merged from fullstack_asterix_stabilization to fullstack_lsm_staging -r3100:3171

git-svn-id: https://hyracks.googlecode.com/svn/branches/fullstack_lsm_staging@3173 123451ca-8445-de46-9d55-352943316053
diff --git a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index edc1b66..2b24bd0 100644
--- a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import edu.uci.ics.hyracks.api.job.IJobletEventListenerFactory;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public class HeuristicCompilerFactoryBuilder extends AbstractCompilerFactoryBuilder {
@@ -81,7 +82,8 @@
                     }
 
                     @Override
-                    public JobSpecification createJob(Object appContext) throws AlgebricksException {
+                    public JobSpecification createJob(Object appContext,
+                            IJobletEventListenerFactory jobEventListenerFactory) throws AlgebricksException {
                         AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Starting Job Generation.\n");
                         JobGenContext context = new JobGenContext(null, metadata, appContext,
                                 serializerDeserializerProvider, hashFunctionFactoryProvider,
@@ -91,7 +93,7 @@
                                 expressionTypeComputer, nullableTypeComputer, oc, expressionEvalSizeComputer,
                                 partialAggregationTypeComputer, frameSize, clusterLocations);
                         PlanCompiler pc = new PlanCompiler(context);
-                        return pc.compilePlan(plan, null);
+                        return pc.compilePlan(plan, null, jobEventListenerFactory);
                     }
                 };
             }
diff --git a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompiler.java b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompiler.java
index 9146722..517ca6b 100644
--- a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompiler.java
+++ b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/ICompiler.java
@@ -15,10 +15,12 @@
 package edu.uci.ics.hyracks.algebricks.compiler.api;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.job.IJobletEventListenerFactory;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 public interface ICompiler {
     public void optimize() throws AlgebricksException;
 
-    public JobSpecification createJob(Object appContext) throws AlgebricksException;
+    public JobSpecification createJob(Object appContext, IJobletEventListenerFactory jobEventListenerFactory)
+            throws AlgebricksException;
 }
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 32cfb9a..0efb5ff 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
@@ -43,7 +43,9 @@
     WRITE_RESULT,
     INSERT_DELETE,
     INDEX_INSERT_DELETE,
-	UPDATE,
+    UPDATE,
     INVERTED_INDEX_SEARCH,
-    PARTITIONINGSPLIT
+    FUZZY_INVERTED_INDEX_SEARCH,
+    PARTITIONINGSPLIT,
+    EXTENSION_OPERATOR
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 82187e3..d516d57 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -60,12 +60,14 @@
             JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<S> dataSource,
-            IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
-            RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
+            IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
+            LogicalVariable payLoadVar, RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec)
+            throws AlgebricksException;
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<S> dataSource,
-            IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
-            RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
+            IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
+            LogicalVariable payLoadVar, RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec)
+            throws AlgebricksException;
 
     /**
      * Creates the insert runtime of IndexInsertDeletePOperator, which models
@@ -140,4 +142,5 @@
     public IDataSourceIndex<I, S> findDataSourceIndex(I indexId, S dataSourceId) throws AlgebricksException;
 
     public IFunctionInfo lookupFunction(FunctionIdentifier fid);
+
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
index c5f4c71..545d039 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
@@ -104,7 +104,7 @@
         if (p.getRoots().size() > 1) {
             throw new NotImplementedException("Nested plans with several roots are not supported.");
         }
-        JobSpecification nestedJob = pc.compilePlan(p, outerPlanSchema);
+        JobSpecification nestedJob = pc.compilePlan(p, outerPlanSchema, null);
         ILogicalOperator topOpInSubplan = p.getRoots().get(0).getValue();
         JobGenContext context = pc.getContext();
         IOperatorSchema topOpInSubplanScm = context.getSchema(topOpInSubplan);
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
index 563fcc5..b09c194 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
@@ -28,7 +28,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -87,7 +86,8 @@
             Object t = env.getVarType(v);
             comparatorFactories[i++] = bcfp.getBinaryComparatorFactory(t, true);
         }
-        RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+        RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
+                propagatedSchema, context);
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
         IOperatorDescriptor opDesc = null;
 
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
index 477d257..3e2827b 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
@@ -11,6 +11,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -70,17 +71,19 @@
             throws AlgebricksException {
         InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op;
         IMetadataProvider mp = context.getMetadataProvider();
+        IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
         JobSpecification spec = builder.getJobSpec();
-        RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0],
-                context);
+        RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
+                context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
 
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
-        if (insertDeleteOp.getOperation() == Kind.INSERT)
-            runtimeAndConstraints = mp.getInsertRuntime(dataSource, propagatedSchema, keys, payload, inputDesc,
-                    context, spec);
-        else
-            runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, keys, payload, inputDesc,
-                    context, spec);
+        if (insertDeleteOp.getOperation() == Kind.INSERT) {
+            runtimeAndConstraints = mp.getInsertRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
+                    inputDesc, context, spec);
+        } else {
+            runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
+                    inputDesc, context, spec);
+        }
 
         builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
         builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
index d153f90..5e8b59e 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
@@ -57,8 +57,13 @@
 import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
 
 /**
+ * <<<<<<< .working
  * Left input is broadcast and preserves its local properties. Right input can
  * be partitioned in any way.
+ * =======
+ * Left input is broadcast and preserves its local properties.
+ * Right input can be partitioned in any way.
+ * >>>>>>> .merge-right.r3014
  */
 public class NLJoinPOperator extends AbstractJoinPOperator {
 
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
index 52f8e0b..a4642bb 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
@@ -61,7 +61,8 @@
         StringStreamingScriptDescription sssd = (StringStreamingScriptDescription) scriptDesc;
         StringStreamingRuntimeFactory runtime = new StringStreamingRuntimeFactory(sssd.getCommand(),
                 sssd.getPrinterFactories(), sssd.getFieldDelimiter(), sssd.getParserFactory());
-        RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+        RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema,
+                context);
         builder.contributeMicroOperator(scriptOp, runtime, recDesc);
         // and contribute one edge from its child
         ILogicalOperator src = scriptOp.getInputs().get(0).getValue();
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/PlanCompiler.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/PlanCompiler.java
index 63a6852..b77d65d 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/PlanCompiler.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/PlanCompiler.java
@@ -27,6 +27,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.api.job.IJobletEventListenerFactory;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
@@ -42,8 +43,11 @@
         return context;
     }
 
-    public JobSpecification compilePlan(ILogicalPlan plan, IOperatorSchema outerPlanSchema) throws AlgebricksException {
+    public JobSpecification compilePlan(ILogicalPlan plan, IOperatorSchema outerPlanSchema, IJobletEventListenerFactory jobEventListenerFactory) throws AlgebricksException {
         JobSpecification spec = new JobSpecification();
+        if (jobEventListenerFactory != null) {
+            spec.setJobletEventListenerFactory(jobEventListenerFactory);
+        }
         List<ILogicalOperator> rootOps = new ArrayList<ILogicalOperator>();
         IHyracksJobBuilder builder = new JobBuilder(spec, context.getClusterLocations());
         for (Mutable<ILogicalOperator> opRef : plan.getRoots()) {
diff --git a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
index 2981157..6c2700d 100644
--- a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
+++ b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
@@ -163,7 +163,7 @@
             LOGGER.info("Optimized Plan:");
             LOGGER.info(getPrettyPrintedPlan(plan));
         }
-        return compiler.createJob(null);
+        return compiler.createJob(null, null);
     }
 
     private ILogicalPlan translate(List<ASTNode> ast) throws PigletException {
diff --git a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
index 15b290e..498b43a 100644
--- a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
+++ b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
@@ -62,13 +62,10 @@
 
     @SuppressWarnings("unchecked")
     @Override
-	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(
-			IDataSource<String> dataSource,
-			List<LogicalVariable> scanVariables,
-			List<LogicalVariable> projectVariables, boolean projectPushed,
-			IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-			JobGenContext context, JobSpecification jobSpec)
-			throws AlgebricksException {
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<String> dataSource,
+            List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
+            IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec)
+            throws AlgebricksException {
         PigletFileDataSource ds = (PigletFileDataSource) dataSource;
 
         FileSplit[] fileSplits = ds.getFileSplits();
@@ -160,22 +157,6 @@
     }
 
     @Override
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<String> dataSource,
-            IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
-            RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<String> dataSource,
-            IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
-            RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(
             IDataSourceIndex<String, String> dataSource, IOperatorSchema propagatedSchema,
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
@@ -194,9 +175,28 @@
         // TODO Auto-generated method stub
         return null;
     }
-    
+
     @Override
     public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
         return FN_MAP.get(fid);
     }
+
+    @Override
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<String> dataSource,
+            IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
+            LogicalVariable payLoadVar, RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec)
+            throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<String> dataSource,
+            IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
+            LogicalVariable payLoadVar, RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec)
+            throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
 }
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 e075f09..53c9d42 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
@@ -258,7 +258,7 @@
         builder.setTypeTraitProvider(HiveTypeTraitProvider.INSTANCE);

         builder.setHashFunctionFamilyProvider(HiveBinaryHashFunctionFamilyProvider.INSTANCE);

 

-        jobSpec = compiler.createJob(null);

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

 

         // set the policy

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

diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
index daf6a7f..fc046fb 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
@@ -8,8 +8,6 @@
 import org.apache.hadoop.hive.ql.plan.PartitionDesc;

 

 import edu.uci.ics.hivesterix.logical.expression.HiveFunctionInfo;

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

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

 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;

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

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

@@ -79,17 +77,19 @@
     }

 

     @Override

-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<S> arg0,

-            IOperatorSchema arg1, List<LogicalVariable> arg2, LogicalVariable arg3, RecordDescriptor arg4,

-            JobGenContext arg5, JobSpecification arg6) throws AlgebricksException {

+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<S> dataSource,

+            IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,

+            LogicalVariable payLoadVar, RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec)

+            throws AlgebricksException {

         // TODO Auto-generated method stub

         return null;

     }

 

     @Override

-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<S> arg0,

-            IOperatorSchema arg1, List<LogicalVariable> arg2, LogicalVariable arg3, RecordDescriptor arg4,

-            JobGenContext arg5, JobSpecification arg6) throws AlgebricksException {

+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<S> dataSource,

+            IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,

+            LogicalVariable payLoadVar, RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec)

+            throws AlgebricksException {

         // TODO Auto-generated method stub

         return null;

     }

@@ -133,5 +133,4 @@
         // TODO Auto-generated method stub

         return null;

     }

-

 }

diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ILinearizeComparator.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ILinearizeComparator.java
new file mode 100644
index 0000000..51a8cfe
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ILinearizeComparator.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2009-2010 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.api.dataflow.value;
+
+public interface ILinearizeComparator extends IBinaryComparator {
+    public int getDimensions();
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ILinearizeComparatorFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ILinearizeComparatorFactory.java
new file mode 100644
index 0000000..79619c4
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/ILinearizeComparatorFactory.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2009-2010 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.api.dataflow.value;
+
+
+public interface ILinearizeComparatorFactory extends IBinaryComparatorFactory {
+    public ILinearizeComparator createBinaryComparator();
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
index 9e27077..ffe41d0 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
@@ -21,19 +21,26 @@
     private static final long serialVersionUID = 1L;
 
     private final File file;
+    private final IODeviceHandle dev;
 
     public FileReference(IODeviceHandle dev, String devRelPath) {
         file = new File(dev.getPath(), devRelPath);
+        this.dev = dev;
     }
 
     public FileReference(File file) {
         this.file = file;
+        this.dev = null;
     }
 
     public File getFile() {
-        return file;
+    	return file;
     }
 
+    public IODeviceHandle getDeviceHandle() {
+    	return dev;
+    }
+    
     @Override
     public String toString() {
         return file.getAbsolutePath();
diff --git a/hyracks/hyracks-control/hyracks-control-common/pom.xml b/hyracks/hyracks-control/hyracks-control-common/pom.xml
index ce1298e..9879d67 100644
--- a/hyracks/hyracks-control/hyracks-control-common/pom.xml
+++ b/hyracks/hyracks-control/hyracks-control-common/pom.xml
@@ -1,9 +1,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>hyracks-control-common</artifactId>
   <name>hyracks-control-common</name>
-  <version>0.2.3-SNAPSHOT</version>
 
   <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/MurmurHash3BinaryHashFunctionFamily.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/MurmurHash3BinaryHashFunctionFamily.java
new file mode 100644
index 0000000..8f2e32c
--- /dev/null
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/MurmurHash3BinaryHashFunctionFamily.java
@@ -0,0 +1,61 @@
+package edu.uci.ics.hyracks.data.std.accessors;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+
+public class MurmurHash3BinaryHashFunctionFamily implements IBinaryHashFunctionFamily {
+    private static final long serialVersionUID = 1L;
+
+    private static final int C1 = 0xcc9e2d51;
+    private static final int C2 = 0x1b873593;
+    private static final int C3 = 5;
+    private static final int C4 = 0xe6546b64;
+    private static final int C5 = 0x85ebca6b;
+    private static final int C6 = 0xc2b2ae35;
+
+    @Override
+    public IBinaryHashFunction createBinaryHashFunction(final int seed) {
+        return new IBinaryHashFunction() {
+            @Override
+            public int hash(byte[] bytes, int offset, int length) {
+                int h = seed;
+                int p = offset;
+                int remain = length;
+                while (remain > 4) {
+                    int k = ((int) bytes[p]) | (((int) bytes[p + 1]) << 8) | (((int) bytes[p + 2]) << 16)
+                            | (((int) bytes[p + 3]) << 24);
+                    k *= C1;
+                    k = Integer.rotateLeft(k, 15);
+                    k *= C2;
+                    h ^= k;
+                    h = Integer.rotateLeft(h, 13);
+                    h = h * C3 + C4;
+                    p += 4;
+                    remain -= 4;
+                }
+                int k = 0;
+                switch (remain) {
+                    case 3:
+                        k = bytes[p++];
+                    case 2:
+                        k = (k << 8) | bytes[p++];
+                    case 1:
+                        k = (k << 8) | bytes[p++];
+                        k *= C1;
+                        k = Integer.rotateLeft(k, 15);
+                        k *= C2;
+                        h ^= k;
+                        h = Integer.rotateLeft(h, 13);
+                        h = h * C3 + C4;
+                }
+                h ^= length;
+                h ^= (h >>> 16);
+                h *= C5;
+                h ^= (h >>> 13);
+                h *= C6;
+                h ^= (h >>> 16);
+                return h;
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/DoublePointable.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/DoublePointable.java
index 5267086..543031d 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/DoublePointable.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/DoublePointable.java
@@ -23,6 +23,17 @@
 import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
 
 public final class DoublePointable extends AbstractPointable implements IHashable, IComparable, INumeric {
+	private final static double machineEpsilon;
+	static {
+		float epsilon = 1.0f;
+
+        do {
+           epsilon /= 2.0f;
+        }
+        while ((float)(1.0 + (epsilon/2.0)) != 1.0);
+        machineEpsilon = epsilon;
+	}
+	
     public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
         private static final long serialVersionUID = 1L;
 
@@ -134,4 +145,8 @@
     public double doubleValue() {
         return getDouble();
     }
+
+	public static double getEpsilon() {
+		return machineEpsilon;
+	}
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/ShortSerializerDeserializer.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/ShortSerializerDeserializer.java
new file mode 100644
index 0000000..15384b3
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/ShortSerializerDeserializer.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2010 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.dataflow.common.data.marshalling;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ShortSerializerDeserializer implements ISerializerDeserializer<Short> {
+    private static final long serialVersionUID = 1L;
+
+    public static final ShortSerializerDeserializer INSTANCE = new ShortSerializerDeserializer();
+
+    private ShortSerializerDeserializer() {
+    }
+
+    @Override
+    public Short deserialize(DataInput in) throws HyracksDataException {
+        try {
+            return in.readShort();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(Short instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeShort(instance.intValue());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public static short getShort(byte[] bytes, int offset) {
+        return (short) (((bytes[offset] & 0xff) << 8) + ((bytes[offset + 1] & 0xff)));
+    }
+
+    public static void putShort(int val, byte[] bytes, int offset) {
+        bytes[offset] = (byte) ((val >>> 8) & 0xFF);
+        bytes[offset + 1] = (byte) ((val >>> 0) & 0xFF);
+    }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
index 00575f4..9dafa83 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
@@ -25,43 +25,45 @@
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.BooleanSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 
 @SuppressWarnings("rawtypes")
 public class SerdeUtils {
-	public static class PayloadTypeTraits implements ITypeTraits {
-		private static final long serialVersionUID = 1L;
-		final int payloadSize;
-		
-		public PayloadTypeTraits(int payloadSize) {
-			this.payloadSize = payloadSize;
-		}
-		
-		@Override
-		public boolean isFixedLength() {
-			return true;
-		}
+    public static class PayloadTypeTraits implements ITypeTraits {
+        private static final long serialVersionUID = 1L;
+        final int payloadSize;
 
-		@Override
-		public int getFixedLength() {
-			return payloadSize;
-		}
-	}
-	
-	public static ITypeTraits[] serdesToTypeTraits(ISerializerDeserializer[] serdes) {
+        public PayloadTypeTraits(int payloadSize) {
+            this.payloadSize = payloadSize;
+        }
+
+        @Override
+        public boolean isFixedLength() {
+            return true;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return payloadSize;
+        }
+    }
+
+    public static ITypeTraits[] serdesToTypeTraits(ISerializerDeserializer[] serdes) {
         ITypeTraits[] typeTraits = new ITypeTraits[serdes.length];
         for (int i = 0; i < serdes.length; i++) {
             typeTraits[i] = serdeToTypeTrait(serdes[i]);
         }
         return typeTraits;
     }
-    
+
     public static ITypeTraits[] serdesToTypeTraits(ISerializerDeserializer[] serdes, int payloadSize) {
         ITypeTraits[] typeTraits = new ITypeTraits[serdes.length + 1];
         for (int i = 0; i < serdes.length; i++) {
@@ -72,6 +74,9 @@
     }
 
     public static ITypeTraits serdeToTypeTrait(ISerializerDeserializer serde) {
+        if (serde instanceof ShortSerializerDeserializer) {
+            return ShortPointable.TYPE_TRAITS;
+        }
         if (serde instanceof IntegerSerializerDeserializer) {
             return IntegerPointable.TYPE_TRAITS;
         }
@@ -112,6 +117,9 @@
     }
 
     public static IBinaryComparatorFactory serdeToComparatorFactory(ISerializerDeserializer serde) {
+        if (serde instanceof ShortSerializerDeserializer) {
+            return PointableBinaryComparatorFactory.of(ShortPointable.FACTORY);
+        }
         if (serde instanceof IntegerSerializerDeserializer) {
             return PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
         }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java
index b35dd75..02047e8 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java
@@ -126,4 +126,11 @@
         tupleCopy.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
         return tupleCopy;
     }
+    
+    public static void copyTuple(ArrayTupleBuilder tupleBuilder, ITupleReference tuple, int numFields) throws HyracksDataException {
+        tupleBuilder.reset();
+        for (int i = 0; i < numFields; i++) {
+            tupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+        }
+    }
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index d86f1d5..e1fda74 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -32,7 +32,7 @@
 import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
 
 public class InMemoryHashJoin {
-	
+
     private final List<ByteBuffer> buffers;
     private final FrameTupleAccessor accessorBuild;
     private final ITuplePartitionComputer tpcBuild;
@@ -44,24 +44,25 @@
     private final boolean isLeftOuter;
     private final ArrayTupleBuilder nullTupleBuild;
     private final ISerializableTable table;
-	private final int tableSize;
+    private final int tableSize;
     private final TuplePointer storedTuplePointer;
-    private final boolean reverseOutputOrder;	//Should we reverse the order of tuples, we are writing in output
-    
+    private final boolean reverseOutputOrder; //Should we reverse the order of tuples, we are writing in output
+
     public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
             ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
-            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1, ISerializableTable table)
-            throws HyracksDataException {
-    	this(ctx, tableSize, accessor0, tpc0, accessor1, tpc1, comparator, isLeftOuter, nullWriters1, table, false);
+            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1,
+            ISerializableTable table) throws HyracksDataException {
+        this(ctx, tableSize, accessor0, tpc0, accessor1, tpc1, comparator, isLeftOuter, nullWriters1, table, false);
     }
-    
+
     public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
             ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
-            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1, ISerializableTable table, boolean reverse) throws HyracksDataException {
-    	this.tableSize = tableSize;
-       	this.table = table;
-       	storedTuplePointer = new TuplePointer();
-       	buffers = new ArrayList<ByteBuffer>();
+            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1,
+            ISerializableTable table, boolean reverse) throws HyracksDataException {
+        this.tableSize = tableSize;
+        this.table = table;
+        storedTuplePointer = new TuplePointer();
+        buffers = new ArrayList<ByteBuffer>();
         this.accessorBuild = accessor1;
         this.tpcBuild = tpc1;
         this.accessorProbe = accessor0;
@@ -82,7 +83,7 @@
         } else {
             nullTupleBuild = null;
         }
-    	reverseOutputOrder = reverse;
+        reverseOutputOrder = reverse;
     }
 
     public void build(ByteBuffer buffer) throws HyracksDataException {
@@ -120,8 +121,8 @@
             } while (true);
 
             if (!matchFound && isLeftOuter) {
-                
-            	if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
+
+                if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
                         nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
                     flushFrame(outBuffer, writer);
                     appender.reset(outBuffer, true);
@@ -130,7 +131,7 @@
                         throw new IllegalStateException();
                     }
                 }
-                
+
             }
         }
     }
@@ -148,25 +149,24 @@
         buffer.position(0);
         buffer.limit(buffer.capacity());
     }
-    
-    private void appendToResult(int probeSidetIx, int buildSidetIx, IFrameWriter writer) throws HyracksDataException{
-    	if(!reverseOutputOrder){
-    		if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
+
+    private void appendToResult(int probeSidetIx, int buildSidetIx, IFrameWriter writer) throws HyracksDataException {
+        if (!reverseOutputOrder) {
+            if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
                 flushFrame(outBuffer, writer);
                 appender.reset(outBuffer, true);
                 if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
                     throw new IllegalStateException();
                 }
             }
-    	}
-    	else{
-    		if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
+        } else {
+            if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
                 flushFrame(outBuffer, writer);
                 appender.reset(outBuffer, true);
                 if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
                     throw new IllegalStateException();
                 }
             }
-    	}
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index 8785f16..a939a04 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -41,15 +41,14 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+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.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+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.common.IStorageManagerInterface;
 
 // This example will insert tuples into the primary and secondary index using an insert pipeline
@@ -121,7 +120,7 @@
         // run data generator on first nodecontroller given
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dataGen, splitNCs[0]);
 
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
+        IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // prepare insertion into primary index
@@ -148,9 +147,9 @@
 
         // create operator descriptor
         TreeIndexInsertUpdateDeleteOperatorDescriptor primaryInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, recDesc, storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, primaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory, null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, recDesc, storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, null, primaryFieldPermutation, IndexOperation.INSERT,
+                dataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE);
         JobHelper.createPartitionConstraint(spec, primaryInsert, splitNCs);
 
         // prepare insertion into secondary index
@@ -172,9 +171,9 @@
                 options.secondaryBTreeName);
         // create operator descriptor
         TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, recDesc, storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
-                secondaryComparatorFactories, secondaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory, null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, recDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, null, secondaryFieldPermutation, IndexOperation.INSERT,
+                dataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE);
         JobHelper.createPartitionConstraint(spec, secondaryInsert, splitNCs);
 
         // end the insert pipeline at this sink operator
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 e9c3b1c..616fdae 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
@@ -40,14 +40,13 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+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.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 // This example will load a primary index from randomly generated data
@@ -138,7 +137,7 @@
         typeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
 
         // create providers for B-Tree
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
+        IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // the B-Tree expects its keyfields to be at the front of its input
@@ -149,8 +148,8 @@
         IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.btreeName);
         IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, btreeSplitProvider, typeTraits, comparatorFactories,
-                fieldPermutation, 0.7f, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, lcManagerProvider, btreeSplitProvider, typeTraits, comparatorFactories, null,
+                fieldPermutation, 0.7f, false, 1000L, 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/PrimaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index f9e567c..3493947 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -38,14 +38,13 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 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.dataflow.IIndex;
+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.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 // This example will perform an ordered scan on the primary index
@@ -100,7 +99,7 @@
         comparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
         // create roviders for B-Tree
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
+        IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // schema of tuples coming out of primary index
@@ -139,8 +138,8 @@
         IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.btreeName);
         IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
         BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, storageManager,
-                indexRegistryProvider, btreeSplitProvider, typeTraits, comparatorFactories, lowKeyFields,
-                highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+                lcManagerProvider, btreeSplitProvider, typeTraits, comparatorFactories, null, lowKeyFields,
+                highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
         JobHelper.createPartitionConstraint(spec, btreeSearchOp, splitNCs);
 
         // have each node print the results of its respective B-Tree
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 bb3e174..075ea20 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
@@ -33,15 +33,14 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+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.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDiskOrderScanOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 // This example will load a secondary index with <key, primary-index key> pairs
@@ -90,7 +89,7 @@
 
         String[] splitNCs = options.ncs.split(",");
 
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
+        IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // schema of tuples that we are retrieving from the primary index
@@ -119,8 +118,8 @@
         IFileSplitProvider primarySplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.primaryBTreeName);
         IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
         TreeIndexDiskOrderScanOperatorDescriptor btreeScanOp = new TreeIndexDiskOrderScanOperatorDescriptor(spec,
-                recDesc, storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                recDesc, storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits,
+                dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         JobHelper.createPartitionConstraint(spec, btreeScanOp, splitNCs);
 
         // sort the tuples as preparation for bulk load into secondary index
@@ -141,8 +140,8 @@
         int[] fieldPermutation = { 1, 0 };
         IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.secondaryBTreeName);
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, btreeSplitProvider, secondaryTypeTraits, comparatorFactories,
-                fieldPermutation, 0.7f, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, lcManagerProvider, btreeSplitProvider, secondaryTypeTraits, comparatorFactories, null,
+                fieldPermutation, 0.7f, false, 1000L, dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
 
         // connect the ops
diff --git a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 1b4bb40..a6ffc13 100644
--- a/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -38,14 +38,13 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
 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.dataflow.IIndex;
+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.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 // This example will perform range search on the secondary index
@@ -91,7 +90,7 @@
 
         String[] splitNCs = options.ncs.split(",");
 
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
+        IIndexLifecycleManagerProvider lcManagerProvider = IndexLifecycleManagerProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // schema of tuples coming out of secondary index
@@ -165,9 +164,9 @@
                 options.secondaryBTreeName);
         IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
         BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
-                storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
-                searchComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
-                dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                searchComparatorFactories, null, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
+                dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
         JobHelper.createPartitionConstraint(spec, secondarySearchOp, splitNCs);
 
         // secondary index will output tuples with [UTF8String, Integer]
@@ -181,9 +180,9 @@
 
         IFileSplitProvider primarySplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.primaryBTreeName);
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
-                dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                null, primaryLowKeyFields, primaryHighKeyFields, true, true, dataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
         JobHelper.createPartitionConstraint(spec, primarySearchOp, splitNCs);
 
         // have each node print the results of its respective B-Tree
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/IndexLifecycleManagerProvider.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/IndexLifecycleManagerProvider.java
new file mode 100644
index 0000000..6a17755
--- /dev/null
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/IndexLifecycleManagerProvider.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.hyracks.examples.btree.helper;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+
+public enum IndexLifecycleManagerProvider implements IIndexLifecycleManagerProvider {
+    INSTANCE;
+
+    @Override
+    public IIndexLifecycleManager getLifecycleManager(IHyracksTaskContext ctx) {
+        return RuntimeContext.get(ctx).getIndexLifecycleManager();
+    }
+
+}
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/IndexRegistryProvider.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/IndexRegistryProvider.java
deleted file mode 100644
index d1f40d8..0000000
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/IndexRegistryProvider.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Copyright 2009-2010 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.examples.btree.helper;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-
-public class IndexRegistryProvider implements IIndexRegistryProvider<IIndex> {
-    private static final long serialVersionUID = 1L;
-
-    public static final IndexRegistryProvider INSTANCE = new IndexRegistryProvider();
-
-    private IndexRegistryProvider() {
-    }
-
-    @Override
-    public IndexRegistry<IIndex> getRegistry(IHyracksTaskContext ctx) {
-        return RuntimeContext.get(ctx).getIndexRegistry();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
index 56d517f..aa5ce1d 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
@@ -17,8 +17,9 @@
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+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.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.buffercache.DelayPageCleanerPolicy;
@@ -28,20 +29,30 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.storage.common.smi.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceRepositoryFactory;
 
 public class RuntimeContext {
-    private IndexRegistry<IIndex> indexRegistry;
     private IBufferCache bufferCache;
     private IFileMapManager fileMapManager;
+    private ILocalResourceRepository localResourceRepository;
+    private IIndexLifecycleManager lcManager;
+    private ResourceIdFactory resourceIdFactory;
 
-    public RuntimeContext(INCApplicationContext appCtx) {
+    public RuntimeContext(INCApplicationContext appCtx) throws HyracksDataException {
         fileMapManager = new TransientFileMapManager();
         ICacheMemoryAllocator allocator = new HeapBufferAllocator();
         IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
         bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), allocator, prs,
                 new DelayPageCleanerPolicy(1000), fileMapManager, 32768, 50, 100);
-        indexRegistry = new IndexRegistry<IIndex>();
+        lcManager = new IndexLifecycleManager();
+        ILocalResourceRepositoryFactory localResourceRepositoryFactory = new TransientLocalResourceRepositoryFactory();
+        localResourceRepository = localResourceRepositoryFactory.createRepository();
+        resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
     }
 
     public void close() {
@@ -56,11 +67,19 @@
         return fileMapManager;
     }
 
-    public IndexRegistry<IIndex> getIndexRegistry() {
-        return indexRegistry;
-    }
-
     public static RuntimeContext get(IHyracksTaskContext ctx) {
         return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
     }
+
+    public ILocalResourceRepository getLocalResourceRepository() {
+        return localResourceRepository;
+    }
+
+    public ResourceIdFactory getResourceIdFactory() {
+        return resourceIdFactory;
+    }
+
+    public IIndexLifecycleManager getIndexLifecycleManager() {
+        return lcManager;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
index 4c6363b..e19847e 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
@@ -19,6 +19,8 @@
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 
 public class StorageManagerInterface implements IStorageManagerInterface {
     private static final long serialVersionUID = 1L;
@@ -37,4 +39,14 @@
     public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
         return RuntimeContext.get(ctx).getFileMapManager();
     }
+
+    @Override
+    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
+        return RuntimeContext.get(ctx).getLocalResourceRepository();
+    }
+
+    @Override
+    public ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
+        return RuntimeContext.get(ctx).getResourceIdFactory();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/pom.xml b/hyracks/hyracks-examples/btree-example/pom.xml
index d8a4367..6614a6e 100644
--- a/hyracks/hyracks-examples/btree-example/pom.xml
+++ b/hyracks/hyracks-examples/btree-example/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks.examples</groupId>
   <artifactId>btree-example</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>btree-example</name>
 
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/data/orders-with-locations-part1.txt b/hyracks/hyracks-examples/hyracks-integration-tests/data/orders-with-locations-part1.txt
new file mode 100644
index 0000000..e1d423f
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/data/orders-with-locations-part1.txt
@@ -0,0 +1,557 @@
+1|37|O|131251.81|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |42.3631|-71.065|42.3631|-71.065|
+2|79|O|40183.29|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|42.1091|-70.696|42.1091|-70.696|
+3|124|F|160882.76|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|40.8151|-73.0452|40.8151|-73.0452|
+4|137|O|31084.79|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro|40.8151|-73.0452|40.8151|-73.0452|
+5|46|F|86615.25|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly|42.2481|-71.174|42.2481|-71.174|
+6|56|F|36468.55|1992-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia|61.1201|-149.89|61.1201|-149.89|
+7|40|O|171488.73|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests |61.1501|-149.926|61.1501|-149.926|
+32|131|O|116923.00|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep|61.181|-149.814|61.181|-149.814|
+33|67|F|99798.76|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request|61.1517|-149.86|61.1517|-149.86|
+34|62|O|41670.02|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe|61.1806|-149.814|61.1806|-149.814|
+35|128|O|148789.52|1995-10-23|4-NOT SPECIFIED|Clerk#000000259|0|zzle. carefully enticing deposits nag furio|61.1806|-149.775|61.1806|-149.775|
+36|116|O|38988.98|1995-11-03|1-URGENT|Clerk#000000358|0| quick packages are blithely. slyly silent accounts wake qu|61.1806|-149.775|61.1806|-149.775|
+37|88|F|113701.89|1992-06-03|3-MEDIUM|Clerk#000000456|0|kly regular pinto beans. carefully unusual waters cajole never|61.1806|-149.775|61.1806|-149.775|
+38|125|O|46366.56|1996-08-21|4-NOT SPECIFIED|Clerk#000000604|0|haggle blithely. furiously express ideas haggle blithely furiously regular re|61.2113|-149.824|61.2113|-149.824|
+39|82|O|219707.84|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir|61.1967|-149.877|61.1967|-149.877|
+64|34|F|20065.73|1994-07-16|3-MEDIUM|Clerk#000000661|0|wake fluffily. sometimes ironic pinto beans about the dolphin|61.2164|-149.892|61.2164|-149.892|
+65|17|P|65883.92|1995-03-18|1-URGENT|Clerk#000000632|0|ular requests are blithely pending orbits-- even requests against the deposit|61.1571|-149.883|61.1571|-149.883|
+66|130|F|79258.24|1994-01-20|5-LOW|Clerk#000000743|0|y pending requests integrate|61.2048|-149.834|61.2048|-149.834|
+67|58|O|116227.05|1996-12-19|4-NOT SPECIFIED|Clerk#000000547|0|symptotes haggle slyly around the furiously iron|61.0956|-149.843|61.0956|-149.843|
+68|29|O|215135.72|1998-04-18|3-MEDIUM|Clerk#000000440|0| pinto beans sleep carefully. blithely ironic deposits haggle furiously acro|61.1491|-149.809|61.1491|-149.809|
+69|85|F|162176.23|1994-06-04|4-NOT SPECIFIED|Clerk#000000330|0| depths atop the slyly thin deposits detect among the furiously silent accou|61.1981|-149.871|61.1981|-149.871|
+70|65|F|84651.80|1993-12-18|5-LOW|Clerk#000000322|0| carefully ironic request|61.1982|-149.876|61.1982|-149.876|
+71|4|O|178821.73|1998-01-24|4-NOT SPECIFIED|Clerk#000000271|0| express deposits along the blithely regul|61.1924|-149.909|61.1924|-149.909|
+96|109|F|55090.67|1994-04-17|2-HIGH|Clerk#000000395|0|oost furiously. pinto|61.2204|-149.728|61.2204|-149.728|
+97|22|F|68908.31|1993-01-29|3-MEDIUM|Clerk#000000547|0|hang blithely along the regular accounts. furiously even ideas after the|61.196|-149.864|61.196|-149.864|
+98|106|F|51004.44|1994-09-25|1-URGENT|Clerk#000000448|0|c asymptotes. quickly regular packages should have to nag re|61.1987|-149.889|61.1987|-149.889|
+99|89|F|92326.79|1994-03-13|4-NOT SPECIFIED|Clerk#000000973|0|e carefully ironic packages. pending|61.1984|-149.897|61.1984|-149.897|
+100|148|O|141311.01|1998-02-28|4-NOT SPECIFIED|Clerk#000000577|0|heodolites detect slyly alongside of the ent|61.1897|-149.898|61.1897|-149.898|
+101|28|O|95591.40|1996-03-17|3-MEDIUM|Clerk#000000419|0|ding accounts above the slyly final asymptote|61.1228|-149.81|61.1228|-149.81|
+102|1|O|113954.89|1997-05-09|2-HIGH|Clerk#000000596|0| slyly according to the asymptotes. carefully final packages integrate furious|61.1649|-149.881|61.1649|-149.881|
+103|31|O|95563.95|1996-06-20|4-NOT SPECIFIED|Clerk#000000090|0|ges. carefully unusual instructions haggle quickly regular f|61.1934|-149.887|61.1934|-149.887|
+128|74|F|36333.34|1992-06-15|1-URGENT|Clerk#000000385|0|ns integrate fluffily. ironic asymptotes after the regular excuses nag around |61.2164|-149.892|61.2164|-149.892|
+129|73|F|188124.55|1992-11-19|5-LOW|Clerk#000000859|0|ing tithes. carefully pending deposits boost about the silently express |61.1932|-149.886|61.1932|-149.886|
+130|37|F|115717.37|1992-05-08|2-HIGH|Clerk#000000036|0|le slyly unusual, regular packages? express deposits det|61.2072|-149.888|61.2072|-149.888|
+131|94|F|96596.81|1994-06-08|3-MEDIUM|Clerk#000000625|0|after the fluffily special foxes integrate s|61.2125|-149.904|61.2125|-149.904|
+132|28|F|118802.62|1993-06-11|3-MEDIUM|Clerk#000000488|0|sits are daringly accounts. carefully regular foxes sleep slyly about the|61.2142|-149.806|61.2142|-149.806|
+133|44|O|80437.72|1997-11-29|1-URGENT|Clerk#000000738|0|usly final asymptotes |61.1866|-149.923|61.1866|-149.923|
+134|7|F|154260.84|1992-05-01|4-NOT SPECIFIED|Clerk#000000711|0|lar theodolites boos|61.1089|-149.857|61.1089|-149.857|
+135|61|O|174569.88|1995-10-21|4-NOT SPECIFIED|Clerk#000000804|0|l platelets use according t|61.1024|-149.853|61.1024|-149.853|
+160|83|O|86076.86|1996-12-19|4-NOT SPECIFIED|Clerk#000000342|0|thely special sauternes wake slyly of t|61.1891|-149.906|61.1891|-149.906|
+161|17|F|19056.99|1994-08-31|2-HIGH|Clerk#000000322|0|carefully! special instructions sin|61.1891|-149.906|61.1891|-149.906|
+162|16|O|2158.13|1995-05-08|3-MEDIUM|Clerk#000000378|0|nts hinder fluffily ironic instructions. express, express excuses |61.1891|-149.906|61.1891|-149.906|
+163|88|O|125170.86|1997-09-05|3-MEDIUM|Clerk#000000379|0|y final packages. final foxes since the quickly even|61.1891|-149.906|61.1891|-149.906|
+164|1|F|202660.52|1992-10-21|5-LOW|Clerk#000000209|0|cajole ironic courts. slyly final ideas are slyly. blithely final Tiresias sub|61.1891|-149.906|61.1891|-149.906|
+165|28|F|141824.23|1993-01-30|4-NOT SPECIFIED|Clerk#000000292|0|across the blithely regular accounts. bold|61.1891|-149.906|61.1891|-149.906|
+166|109|O|93335.60|1995-09-12|2-HIGH|Clerk#000000440|0|lets. ironic, bold asymptotes kindle|61.1891|-149.906|61.1891|-149.906|
+167|121|F|52982.23|1993-01-04|4-NOT SPECIFIED|Clerk#000000731|0|s nag furiously bold excuses. fluffily iron|61.1891|-149.906|61.1891|-149.906|
+192|83|O|133002.55|1997-11-25|5-LOW|Clerk#000000483|0|y unusual platelets among the final instructions integrate rut|61.1891|-149.906|61.1891|-149.906|
+193|80|F|48053.18|1993-08-08|1-URGENT|Clerk#000000025|0|the furiously final pin|61.1891|-149.906|61.1891|-149.906|
+194|62|F|114097.63|1992-04-05|3-MEDIUM|Clerk#000000352|0|egular requests haggle slyly regular, regular pinto beans. asymptote|61.1891|-149.906|61.1891|-149.906|
+195|136|F|120053.52|1993-12-28|3-MEDIUM|Clerk#000000216|0|old forges are furiously sheaves. slyly fi|61.1891|-149.906|61.1891|-149.906|
+196|65|F|33248.04|1993-03-17|2-HIGH|Clerk#000000988|0|beans boost at the foxes. silent foxes|61.1891|-149.906|61.1891|-149.906|
+197|34|P|100290.07|1995-04-07|2-HIGH|Clerk#000000969|0|solve quickly about the even braids. carefully express deposits affix care|61.1891|-149.906|61.1891|-149.906|
+198|112|O|125792.83|1998-01-02|4-NOT SPECIFIED|Clerk#000000331|0|its. carefully ironic requests sleep. furiously express fox|61.1891|-149.906|61.1891|-149.906|
+199|53|O|80592.44|1996-03-07|2-HIGH|Clerk#000000489|0|g theodolites. special packag|61.1891|-149.906|61.1891|-149.906|
+224|4|F|155680.60|1994-06-18|4-NOT SPECIFIED|Clerk#000000642|0|r the quickly thin courts. carefully|61.1891|-149.906|61.1891|-149.906|
+225|34|P|165890.47|1995-05-25|1-URGENT|Clerk#000000177|0|s. blithely ironic accounts wake quickly fluffily special acc|61.1891|-149.906|61.1891|-149.906|
+226|128|F|180119.22|1993-03-10|2-HIGH|Clerk#000000756|0|s are carefully at the blithely ironic acc|61.1891|-149.906|61.1891|-149.906|
+227|10|O|46076.46|1995-11-10|5-LOW|Clerk#000000919|0| express instructions. slyly regul|61.1891|-149.906|61.1891|-149.906|
+228|46|F|2638.98|1993-02-25|1-URGENT|Clerk#000000562|0|es was slyly among the regular foxes. blithely regular dependenci|61.1891|-149.906|61.1891|-149.906|
+229|112|F|142290.77|1993-12-29|1-URGENT|Clerk#000000628|0|he fluffily even instructions. furiously i|61.1891|-149.906|61.1891|-149.906|
+230|103|F|107231.60|1993-10-27|1-URGENT|Clerk#000000520|0|odolites. carefully quick requ|61.1891|-149.906|61.1891|-149.906|
+231|91|F|141554.06|1994-09-29|2-HIGH|Clerk#000000446|0| packages haggle slyly after the carefully ironic instruct|61.1891|-149.906|61.1891|-149.906|
+256|125|F|106315.25|1993-10-19|4-NOT SPECIFIED|Clerk#000000834|0|he fluffily final ideas might are final accounts. carefully f|61.1891|-149.906|61.1891|-149.906|
+257|124|O|7102.74|1998-03-28|3-MEDIUM|Clerk#000000680|0|ts against the sly warhorses cajole slyly accounts|61.1891|-149.906|61.1891|-149.906|
+258|43|F|186669.10|1993-12-29|1-URGENT|Clerk#000000167|0|dencies. blithely quick packages cajole. ruthlessly final accounts|61.1891|-149.906|61.1891|-149.906|
+259|44|F|75661.70|1993-09-29|4-NOT SPECIFIED|Clerk#000000601|0|ages doubt blithely against the final foxes. carefully express deposits dazzle|61.1891|-149.906|61.1891|-149.906|
+260|106|O|179292.14|1996-12-10|3-MEDIUM|Clerk#000000960|0|lently regular pinto beans sleep after the slyly e|61.1891|-149.906|61.1891|-149.906|
+261|47|F|201003.12|1993-06-29|3-MEDIUM|Clerk#000000310|0|ully fluffily brave instructions. furiousl|61.1891|-149.906|61.1891|-149.906|
+262|31|O|108443.84|1995-11-25|4-NOT SPECIFIED|Clerk#000000551|0|l packages. blithely final pinto beans use carefu|61.1891|-149.906|61.1891|-149.906|
+263|118|F|79782.56|1994-05-17|2-HIGH|Clerk#000000088|0| pending instructions. blithely un|61.1891|-149.906|61.1891|-149.906|
+288|8|O|163794.53|1997-02-21|1-URGENT|Clerk#000000109|0|uriously final requests. even, final ideas det|61.1891|-149.906|61.1891|-149.906|
+289|104|O|131092.67|1997-02-10|3-MEDIUM|Clerk#000000103|0|sily. slyly special excuse|61.1891|-149.906|61.1891|-149.906|
+290|118|F|62814.89|1994-01-01|4-NOT SPECIFIED|Clerk#000000735|0|efully dogged deposits. furiou|61.1891|-149.906|61.1891|-149.906|
+291|142|F|66817.05|1994-03-13|1-URGENT|Clerk#000000923|0|dolites. carefully regular pinto beans cajol|64.8541|-147.813|64.8541|-147.813|
+292|23|F|30783.05|1992-01-13|2-HIGH|Clerk#000000193|0|g pinto beans will have to sleep f|64.8414|-147.606|64.8414|-147.606|
+293|31|F|37248.78|1992-10-02|2-HIGH|Clerk#000000629|0|re bold, ironic deposits. platelets c|64.8371|-147.746|64.8371|-147.746|
+294|52|F|30059.47|1993-07-16|3-MEDIUM|Clerk#000000499|0|kly according to the frays. final dolphins affix quickly |64.8151|-147.707|64.8151|-147.707|
+295|19|F|89345.99|1994-09-29|2-HIGH|Clerk#000000155|0| unusual pinto beans play. regular ideas haggle|64.8371|-147.746|64.8371|-147.746|
+320|1|O|39835.54|1997-11-21|2-HIGH|Clerk#000000573|0|ar foxes nag blithely|64.849|-147.813|64.849|-147.813|
+321|124|F|62251.15|1993-03-21|3-MEDIUM|Clerk#000000289|0|equests run. blithely final dependencies after the deposits wake caref|64.8425|-147.724|64.8425|-147.724|
+322|134|F|127068.89|1992-03-19|1-URGENT|Clerk#000000158|0|fully across the slyly bold packages. packages against the quickly regular i|64.8425|-147.724|64.8425|-147.724|
+323|40|F|79683.42|1994-03-26|1-URGENT|Clerk#000000959|0|arefully pending foxes sleep blithely. slyly express accoun|64.849|-147.826|64.849|-147.826|
+324|106|F|26868.85|1992-03-20|1-URGENT|Clerk#000000352|0| about the ironic, regular deposits run blithely against the excuses|64.815|-147.882|64.815|-147.882|
+325|41|F|71543.41|1993-10-17|5-LOW|Clerk#000000844|0|ly sometimes pending pa|64.8906|-147.628|64.8906|-147.628|
+326|76|O|229165.17|1995-06-04|2-HIGH|Clerk#000000466|0| requests. furiously ironic asymptotes mold carefully alongside of the blit|64.8276|-147.639|64.8276|-147.639|
+327|145|P|24468.16|1995-04-17|5-LOW|Clerk#000000992|0|ng the slyly final courts. slyly even escapades eat |64.8461|-147.813|64.8461|-147.813|
+352|107|F|16003.86|1994-03-08|2-HIGH|Clerk#000000932|0|ke slyly bold pinto beans. blithely regular accounts against the spe|64.8281|-147.812|64.8281|-147.812|
+353|2|F|179984.42|1993-12-31|5-LOW|Clerk#000000449|0| quiet ideas sleep. even instructions cajole slyly. silently spe|64.8377|-147.718|64.8377|-147.718|
+354|139|O|157062.70|1996-03-14|2-HIGH|Clerk#000000511|0|ly regular ideas wake across the slyly silent ideas. final deposits eat b|64.8417|-147.718|64.8417|-147.718|
+355|71|F|69447.25|1994-06-14|5-LOW|Clerk#000000532|0|s. sometimes regular requests cajole. regular, pending accounts a|64.8145|-147.772|64.8145|-147.772|
+356|148|F|162786.67|1994-06-30|4-NOT SPECIFIED|Clerk#000000944|0|as wake along the bold accounts. even, |64.8541|-147.813|64.8541|-147.813|
+357|61|O|98723.11|1996-10-09|2-HIGH|Clerk#000000301|0|e blithely about the express, final accounts. quickl|64.8169|-147.779|64.8169|-147.779|
+358|4|F|226806.66|1993-09-20|2-HIGH|Clerk#000000392|0|l, silent instructions are slyly. silently even de|64.8378|-147.71|64.8378|-147.71|
+359|79|F|142891.22|1994-12-19|3-MEDIUM|Clerk#000000934|0|n dolphins. special courts above the carefully ironic requests use|64.8436|-147.722|64.8436|-147.722|
+384|115|F|122785.82|1992-03-03|5-LOW|Clerk#000000206|0|, even accounts use furiously packages. slyly ironic pla|64.9401|-147.402|64.9401|-147.402|
+385|34|O|50724.06|1996-03-22|5-LOW|Clerk#000000600|0|hless accounts unwind bold pain|64.8426|-147.719|64.8426|-147.719|
+386|61|F|90380.40|1995-01-25|2-HIGH|Clerk#000000648|0| haggle quickly. stealthily bold asymptotes haggle among the furiously even re|64.8534|-147.811|64.8534|-147.811|
+387|4|O|130647.18|1997-01-26|4-NOT SPECIFIED|Clerk#000000768|0| are carefully among the quickly even deposits. furiously silent req|64.9341|-147.928|64.9341|-147.928|
+388|46|F|120533.46|1992-12-16|4-NOT SPECIFIED|Clerk#000000356|0|ar foxes above the furiously ironic deposits nag slyly final reque|64.8393|-147.72|64.8393|-147.72|
+389|127|F|1984.14|1994-02-17|2-HIGH|Clerk#000000062|0|ing to the regular asymptotes. final, pending foxes about the blithely sil|64.8406|-147.731|64.8406|-147.731|
+390|103|O|168562.27|1998-04-07|5-LOW|Clerk#000000404|0|xpress asymptotes use among the regular, final pinto b|64.9281|-147.865|64.9281|-147.865|
+391|112|F|13282.23|1994-11-17|2-HIGH|Clerk#000000256|0|orges thrash fluffil|64.8371|-147.716|64.8371|-147.716|
+416|41|F|71362.50|1993-09-27|5-LOW|Clerk#000000294|0| the accounts. fluffily bold depo|64.9414|-147.841|64.9414|-147.841|
+417|55|F|91982.29|1994-02-06|3-MEDIUM|Clerk#000000468|0|ironic, even packages. thinly unusual accounts sleep along the slyly unusual |64.8363|-147.79|64.8363|-147.79|
+418|95|P|33124.96|1995-04-13|4-NOT SPECIFIED|Clerk#000000643|0|. furiously ironic instruc|64.8371|-147.716|64.8371|-147.716|
+419|118|O|111597.96|1996-10-01|3-MEDIUM|Clerk#000000376|0|osits. blithely pending theodolites boost carefully|64.8591|-147.917|64.8591|-147.917|
+420|91|O|198039.23|1995-10-31|4-NOT SPECIFIED|Clerk#000000756|0|leep carefully final excuses. fluffily pending requests unwind carefully above|64.8363|-147.79|64.8363|-147.79|
+421|40|F|1084.38|1992-02-22|5-LOW|Clerk#000000405|0|egular, even packages according to the final, un|55.3801|-131.682|55.3801|-131.682|
+422|74|O|106045.89|1997-05-31|4-NOT SPECIFIED|Clerk#000000049|0|aggle carefully across the accounts. regular accounts eat fluffi|55.3073|-131.528|55.3073|-131.528|
+423|104|O|26981.31|1996-06-01|1-URGENT|Clerk#000000674|0|quests. deposits cajole quickly. furiously bold accounts haggle q|55.3801|-131.682|55.3801|-131.682|
+448|149|O|114978.03|1995-08-21|3-MEDIUM|Clerk#000000597|0| regular, express foxes use blithely. quic|55.3601|-131.681|55.3601|-131.681|
+449|97|O|41605.63|1995-07-20|2-HIGH|Clerk#000000841|0|. furiously regular theodolites affix blithely |55.3279|-131.613|55.3279|-131.613|
+450|49|P|153386.61|1995-03-05|4-NOT SPECIFIED|Clerk#000000293|0|d theodolites. boldly bold foxes since the pack|55.3129|-131.588|55.3129|-131.588|
+451|100|O|104664.40|1998-05-25|5-LOW|Clerk#000000048|0|nic pinto beans. theodolites poach carefully; |55.3801|-131.682|55.3801|-131.682|
+452|61|O|2007.48|1997-10-14|1-URGENT|Clerk#000000498|0|t, unusual instructions above the blithely bold pint|55.3801|-131.682|55.3801|-131.682|
+453|46|O|216826.73|1997-05-26|5-LOW|Clerk#000000504|0|ss foxes. furiously regular ideas sleep according to t|55.4299|-131.789|55.4299|-131.789|
+454|49|O|23198.24|1995-12-27|5-LOW|Clerk#000000890|0|dolites sleep carefully blithely regular deposits. quickly regul|55.3801|-131.682|55.3801|-131.682|
+455|13|O|138010.76|1996-12-04|1-URGENT|Clerk#000000796|0| about the final platelets. dependen|55.3507|-131.671|55.3507|-131.671|
+480|73|F|20530.97|1993-05-08|5-LOW|Clerk#000000004|0|ealthy pinto beans. fluffily regular requests along the special sheaves wake |55.3801|-131.682|55.3801|-131.682|
+481|31|F|117827.18|1992-10-08|2-HIGH|Clerk#000000230|0|ly final ideas. packages haggle fluffily|55.3394|-131.636|55.3394|-131.636|
+482|127|O|136634.34|1996-03-26|1-URGENT|Clerk#000000295|0|ts. deposits wake: final acco|55.3801|-131.682|55.3801|-131.682|
+483|35|O|39793.05|1995-07-11|2-HIGH|Clerk#000000025|0|cross the carefully final e|55.3103|-131.582|55.3103|-131.582|
+484|55|O|219920.62|1997-01-03|3-MEDIUM|Clerk#000000545|0|grouches use. furiously bold accounts maintain. bold, regular deposits|55.3801|-131.682|55.3801|-131.682|
+485|101|O|110432.76|1997-03-26|2-HIGH|Clerk#000000105|0| regular ideas nag thinly furiously s|55.3801|-131.682|55.3801|-131.682|
+486|52|O|185968.15|1996-03-11|4-NOT SPECIFIED|Clerk#000000803|0|riously dolphins. fluffily ironic requ|55.3801|-131.682|55.3801|-131.682|
+487|109|F|48502.79|1992-08-18|1-URGENT|Clerk#000000086|0|ithely unusual courts eat accordi|55.3801|-131.682|55.3801|-131.682|
+512|64|P|124661.48|1995-05-20|5-LOW|Clerk#000000814|0|ding requests. carefully express theodolites was quickly. furious|55.3801|-131.682|55.3801|-131.682|
+513|61|O|63703.92|1995-05-01|2-HIGH|Clerk#000000522|0|regular packages. pinto beans cajole carefully against the even|55.3424|-131.634|55.3424|-131.634|
+514|76|O|104585.77|1996-04-04|2-HIGH|Clerk#000000094|0| cajole furiously. slyly final excuses cajole. slyly special instructions |55.4097|-131.729|55.4097|-131.729|
+515|142|F|153720.22|1993-08-29|4-NOT SPECIFIED|Clerk#000000700|0|eposits are furiously furiously silent pinto beans. pending pack|55.3801|-131.682|55.3801|-131.682|
+516|44|O|10677.86|1998-04-21|2-HIGH|Clerk#000000305|0|lar, unusual platelets are carefully. even courts sleep bold, final pinto bea|55.3801|-131.682|55.3801|-131.682|
+517|10|O|82197.79|1997-04-07|5-LOW|Clerk#000000359|0|slyly pending deposits cajole quickly packages. furiou|55.3462|-131.658|55.3462|-131.658|
+518|145|O|223537.09|1998-02-08|2-HIGH|Clerk#000000768|0| the carefully bold accounts. quickly regular excuses are|55.3801|-131.682|55.3801|-131.682|
+519|64|O|95731.50|1997-10-31|1-URGENT|Clerk#000000985|0|ains doze furiously against the f|55.3801|-131.682|55.3801|-131.682|
+544|94|F|47627.89|1993-02-17|2-HIGH|Clerk#000000145|0|the special, final accounts. dogged dolphins|55.3801|-131.682|55.3801|-131.682|
+545|64|O|23476.12|1995-11-07|2-HIGH|Clerk#000000537|0|as. blithely final hockey players about th|55.3801|-131.682|55.3801|-131.682|
+546|145|O|14790.37|1996-11-01|2-HIGH|Clerk#000000041|0|osits sleep. slyly special dolphins about the q|55.3801|-131.682|55.3801|-131.682|
+547|100|O|96855.29|1996-06-22|3-MEDIUM|Clerk#000000976|0|ing accounts eat. carefully regular packa|55.3801|-131.682|55.3801|-131.682|
+548|124|F|99088.75|1994-09-21|1-URGENT|Clerk#000000435|0|arefully express instru|55.3801|-131.682|55.3801|-131.682|
+549|110|F|141679.41|1992-07-13|1-URGENT|Clerk#000000196|0|ideas alongside of |55.3801|-131.682|55.3801|-131.682|
+550|25|O|33123.28|1995-08-02|1-URGENT|Clerk#000000204|0|t requests. blithely |61.5856|-149.316|61.5856|-149.316|
+551|91|O|46355.83|1995-05-30|1-URGENT|Clerk#000000179|0|xpress accounts boost quic|61.5781|-149.429|61.5781|-149.429|
+576|31|O|18307.45|1997-05-13|3-MEDIUM|Clerk#000000955|0|l requests affix regular requests. final account|61.6141|-149.457|61.6141|-149.457|
+577|56|F|34768.68|1994-12-19|5-LOW|Clerk#000000154|0| deposits engage stealthil|61.5801|-149.461|61.5801|-149.461|
+578|94|O|70392.02|1997-01-10|5-LOW|Clerk#000000281|0|e blithely even packages. slyly pending platelets bes|61.9071|-150.067|61.9071|-150.067|
+579|68|O|120828.12|1998-03-11|2-HIGH|Clerk#000000862|0| regular instructions. blithely even p|61.5928|-149.392|61.5928|-149.392|
+580|61|O|88219.12|1997-07-05|2-HIGH|Clerk#000000314|0|tegrate fluffily regular accou|61.6141|-149.457|61.6141|-149.457|
+581|70|O|126066.00|1997-02-23|4-NOT SPECIFIED|Clerk#000000239|0| requests. even requests use slyly. blithely ironic |61.5792|-149.36|61.5792|-149.36|
+582|50|O|129004.81|1997-10-21|1-URGENT|Clerk#000000378|0|n pinto beans print a|61.6049|-149.463|61.6049|-149.463|
+583|49|O|127817.38|1997-03-19|3-MEDIUM|Clerk#000000792|0|efully express requests. a|61.6099|-149.328|61.6099|-149.328|
+608|26|O|62567.99|1996-02-28|3-MEDIUM|Clerk#000000995|0|nic waters wake slyly slyly expre|61.5531|-149.651|61.5531|-149.651|
+609|127|F|21088.59|1994-06-01|3-MEDIUM|Clerk#000000348|0|- ironic gifts believe furiously ca|61.6141|-149.457|61.6141|-149.457|
+610|52|O|175142.28|1995-08-02|1-URGENT|Clerk#000000610|0|totes. ironic, unusual packag|61.6141|-149.457|61.6141|-149.457|
+611|106|F|73907.63|1993-01-27|1-URGENT|Clerk#000000401|0|ounts detect furiously ac|61.5531|-149.651|61.5531|-149.651|
+612|82|F|145695.42|1992-10-21|3-MEDIUM|Clerk#000000759|0|boost quickly quickly final excuses. final foxes use bravely afte|61.6141|-149.457|61.6141|-149.457|
+613|139|O|33396.35|1995-06-18|2-HIGH|Clerk#000000172|0|ts hinder among the deposits. fluffily ironic depos|61.7321|-150.12|61.7321|-150.12|
+614|134|F|218116.21|1992-12-01|2-HIGH|Clerk#000000388|0| deposits! even, daring theodol|61.6141|-149.457|61.6141|-149.457|
+615|67|F|32890.89|1992-05-09|5-LOW|Clerk#000000388|0|t to promise asymptotes. packages haggle alongside of the fluffil|61.582|-149.441|61.582|-149.441|
+640|97|F|145495.62|1993-01-23|2-HIGH|Clerk#000000433|0|r, unusual accounts boost carefully final ideas. slyly silent theod|61.5818|-149.44|61.5818|-149.44|
+641|133|F|120626.49|1993-08-30|5-LOW|Clerk#000000175|0|ents cajole furiously about the quickly silent pac|61.6141|-149.457|61.6141|-149.457|
+642|40|F|22994.51|1993-12-16|3-MEDIUM|Clerk#000000357|0| among the requests wake slyly alongside of th|61.7321|-150.12|61.7321|-150.12|
+643|58|P|180396.95|1995-03-25|2-HIGH|Clerk#000000354|0|g dependencies. regular accounts |61.6308|-149.415|61.6308|-149.415|
+644|8|F|201268.06|1992-05-01|1-URGENT|Clerk#000000550|0| blithely unusual platelets haggle ironic, special excuses. excuses unwi|61.5801|-149.461|61.5801|-149.461|
+645|115|F|234763.73|1994-12-03|2-HIGH|Clerk#000000090|0|quickly daring theodolites across the regu|61.5811|-149.444|61.5811|-149.444|
+646|52|F|142070.65|1994-11-22|2-HIGH|Clerk#000000203|0|carefully even foxes. fina|61.6521|-149.92|61.6521|-149.92|
+647|143|O|56449.23|1997-08-07|1-URGENT|Clerk#000000270|0|egular pearls. carefully express asymptotes are. even account|61.6141|-149.307|61.6141|-149.307|
+672|109|F|89877.09|1994-04-14|5-LOW|Clerk#000000106|0|egular requests are furiously according to |61.6168|-149.328|61.6168|-149.328|
+673|80|F|21137.08|1994-03-10|1-URGENT|Clerk#000000448|0| special pinto beans use quickly furiously even depende|61.5714|-149.381|61.5714|-149.381|
+674|34|F|27204.60|1992-08-29|5-LOW|Clerk#000000448|0|ully special deposits. furiously final warhorses affix carefully. fluffily f|61.6521|-149.92|61.6521|-149.92|
+675|13|O|125188.72|1997-07-31|2-HIGH|Clerk#000000168|0|ffily between the careful|61.5858|-149.376|61.5858|-149.376|
+676|38|O|163966.67|1996-12-13|2-HIGH|Clerk#000000248|0|the final deposits. special, pending|61.5822|-149.463|61.5822|-149.463|
+677|124|F|147915.68|1993-11-24|3-MEDIUM|Clerk#000000824|0|uriously special pinto beans cajole carefully. fi|61.5861|-149.303|61.5861|-149.303|
+678|131|F|135761.05|1993-02-27|5-LOW|Clerk#000000530|0|. blithely final somas about the|61.5821|-149.438|61.5821|-149.438|
+679|49|O|8945.03|1995-12-15|2-HIGH|Clerk#000000853|0|tealthy, final pinto beans haggle slyly. pending platelets about the special, |61.6281|-149.338|61.6281|-149.338|
+704|85|O|56210.26|1996-11-21|3-MEDIUM|Clerk#000000682|0|blithely pending platelets wake alongside of the final, iron|61.5771|-149.335|61.5771|-149.335|
+705|43|O|83773.49|1997-02-13|4-NOT SPECIFIED|Clerk#000000294|0|ithely regular dependencies. express, even packages sleep slyly pending t|61.5917|-149.464|61.5917|-149.464|
+706|148|O|23973.60|1995-09-09|1-URGENT|Clerk#000000448|0|g the packages. deposits caj|61.1927|-149.86|61.1927|-149.86|
+707|118|F|58218.35|1994-11-20|3-MEDIUM|Clerk#000000199|0| ideas about the silent, bold deposits nag dolphins|61.1879|-149.85|61.1879|-149.85|
+708|32|O|100445.59|1998-07-03|3-MEDIUM|Clerk#000000101|0|lphins cajole about t|61.1814|-149.849|61.1814|-149.849|
+709|37|O|72055.87|1998-04-21|1-URGENT|Clerk#000000461|0|ons alongside of the carefully bold pinto bea|61.2104|-149.892|61.2104|-149.892|
+710|133|F|208974.42|1993-01-02|5-LOW|Clerk#000000026|0| regular, regular requests boost. fluffily re|61.2093|-149.903|61.2093|-149.903|
+711|64|F|92484.70|1993-09-23|4-NOT SPECIFIED|Clerk#000000856|0|its. fluffily regular gifts are furi|61.1481|-149.829|61.1481|-149.829|
+736|47|O|130204.17|1998-06-21|5-LOW|Clerk#000000881|0|refully of the final pi|61.2161|-149.876|61.2161|-149.876|
+737|121|F|12984.85|1992-04-26|5-LOW|Clerk#000000233|0|ake blithely express, ironic theodolites. blithely special accounts wa|61.1972|-149.75|61.1972|-149.75|
+738|22|F|114145.18|1993-03-02|4-NOT SPECIFIED|Clerk#000000669|0|ly even foxes. furiously regular accounts cajole ca|61.2066|-149.887|61.2066|-149.887|
+739|1|O|159171.69|1998-05-31|5-LOW|Clerk#000000900|0| against the slyly ironic packages nag slyly ironic|61.2161|-149.876|61.2161|-149.876|
+740|44|O|83490.99|1995-07-16|3-MEDIUM|Clerk#000000583|0|courts haggle furiously across the final, regul|61.195|-149.834|61.195|-149.834|
+741|106|O|47985.98|1998-07-07|2-HIGH|Clerk#000000295|0|ic instructions. slyly express instructions solv|61.2038|-149.808|61.2038|-149.808|
+742|103|F|207632.55|1994-12-23|5-LOW|Clerk#000000543|0|equests? slyly ironic dolphins boost carefully above the blithely|61.1228|-149.862|61.1228|-149.862|
+743|79|O|23614.89|1996-10-04|4-NOT SPECIFIED|Clerk#000000933|0|eans. furiously ironic deposits sleep carefully carefully qui|61.2005|-149.785|61.2005|-149.785|
+768|98|O|220636.82|1996-08-20|3-MEDIUM|Clerk#000000411|0|jole slyly ironic packages. slyly even idea|61.181|-149.825|61.181|-149.825|
+769|80|F|43092.76|1993-06-02|3-MEDIUM|Clerk#000000172|0|ggle furiously. ironic packages haggle slyly. bold platelets affix s|61.1867|-149.919|61.1867|-149.919|
+770|32|O|64271.75|1998-05-23|5-LOW|Clerk#000000572|0|heodolites. furiously special pinto beans cajole pac|61.1955|-149.911|61.1955|-149.911|
+771|46|O|105302.05|1995-06-17|1-URGENT|Clerk#000000105|0|s. furiously final instructions across the deposit|61.1089|-149.858|61.1089|-149.858|
+772|97|F|128234.96|1993-04-17|2-HIGH|Clerk#000000430|0|s boost blithely fluffily idle ideas? fluffily even pin|61.1805|-149.889|61.1805|-149.889|
+773|133|F|146862.27|1993-09-26|3-MEDIUM|Clerk#000000307|0|tions are quickly accounts. accounts use bold, even pinto beans. gifts ag|61.1534|-149.985|61.1534|-149.985|
+774|80|O|145857.60|1995-12-04|1-URGENT|Clerk#000000883|0|tealthily even depths|61.1901|-149.911|61.1901|-149.911|
+775|134|F|59455.61|1995-03-18|1-URGENT|Clerk#000000191|0|kly express requests. fluffily silent accounts poach furiously|61.2122|-149.734|61.2122|-149.734|
+800|56|O|87892.38|1998-07-14|2-HIGH|Clerk#000000213|0|y alongside of the pending packages? final platelets nag fluffily carefu|61.1951|-149.906|61.1951|-149.906|
+801|118|F|127717.72|1992-02-18|1-URGENT|Clerk#000000186|0|iously from the furiously enticing reques|61.2043|-149.869|61.2043|-149.869|
+802|137|F|156381.95|1995-01-05|1-URGENT|Clerk#000000516|0|posits. ironic, pending requests cajole. even theodol|61.2036|-149.869|61.2036|-149.869|
+803|16|O|27629.66|1997-04-29|5-LOW|Clerk#000000260|0|ic instructions. even deposits haggle furiously at the deposits-- regular de|61.1883|-149.886|61.1883|-149.886|
+804|50|F|94400.43|1993-03-12|3-MEDIUM|Clerk#000000931|0|s. blithely final foxes are about the packag|61.2141|-149.864|61.2141|-149.864|
+805|127|O|90042.41|1995-07-05|4-NOT SPECIFIED|Clerk#000000856|0|y according to the fluffily |61.1955|-149.782|61.1955|-149.782|
+806|131|O|26839.16|1996-06-20|2-HIGH|Clerk#000000240|0| the ironic packages wake carefully fina|61.2183|-149.894|61.2183|-149.894|
+807|145|F|222392.53|1993-11-24|3-MEDIUM|Clerk#000000012|0|refully special tithes. blithely regular accoun|61.1417|-149.864|61.1417|-149.864|
+832|29|F|68494.08|1992-04-19|5-LOW|Clerk#000000495|0|xes. bravely regular packages sleep up the furiously bold accou|61.1883|-149.883|61.1883|-149.883|
+833|56|F|49033.69|1994-02-13|3-MEDIUM|Clerk#000000437|0|ts haggle quickly across the slyl|61.2161|-149.876|61.2161|-149.876|
+834|43|F|46459.92|1994-05-23|3-MEDIUM|Clerk#000000805|0| sleep. quickly even foxes are boldly. slyly express requests use slyly|61.2193|-149.869|61.2193|-149.869|
+835|65|O|62430.67|1995-10-08|4-NOT SPECIFIED|Clerk#000000416|0|s about the carefully special foxes haggle quickly about the|61.2191|-149.888|61.2191|-149.888|
+836|70|O|72843.48|1996-11-25|4-NOT SPECIFIED|Clerk#000000729|0|ely bold excuses sleep regular ideas. furiously unusual ideas wake furiou|61.2191|-149.888|61.2191|-149.888|
+837|116|F|60918.41|1994-06-15|4-NOT SPECIFIED|Clerk#000000563|0|kages sleep slyly above the ironic, final orbits|61.2191|-149.888|61.2191|-149.888|
+838|17|O|82918.36|1998-01-29|5-LOW|Clerk#000000213|0| slyly around the slyly even|61.2191|-149.888|61.2191|-149.888|
+839|28|O|70182.63|1995-08-08|1-URGENT|Clerk#000000951|0|the carefully even platelets. furiously unusual fo|61.2191|-149.888|61.2191|-149.888|
+864|139|O|74710.74|1997-08-17|1-URGENT|Clerk#000000036|0|ly after the slyly regular deposits. express, regular asymptotes nag ca|61.2191|-149.888|61.2191|-149.888|
+865|4|F|70430.54|1993-05-04|3-MEDIUM|Clerk#000000337|0|. special packages wake after the carefully final accounts. express pinto be|61.2191|-149.888|61.2191|-149.888|
+866|40|F|4766.19|1992-11-28|3-MEDIUM|Clerk#000000718|0|ins after the even, even accounts nod blithel|61.2191|-149.888|61.2191|-149.888|
+867|26|F|7471.75|1993-11-16|3-MEDIUM|Clerk#000000877|0|pades nag quickly final, |61.2191|-149.888|61.2191|-149.888|
+868|104|F|127345.45|1992-06-09|4-NOT SPECIFIED|Clerk#000000782|0|onic theodolites print carefully. blithely dogge|61.2191|-149.888|61.2191|-149.888|
+869|136|O|58932.19|1997-01-12|2-HIGH|Clerk#000000245|0|ar sheaves are slowly. slyly even attainments boost theodolites. furiously|61.2191|-149.888|61.2191|-149.888|
+870|34|F|40492.37|1993-06-20|4-NOT SPECIFIED|Clerk#000000123|0|blithely ironic ideas nod. sly, r|61.2191|-149.888|61.2191|-149.888|
+871|16|O|172861.58|1995-11-15|5-LOW|Clerk#000000882|0|oss the ironic theodolites.|61.1891|-149.906|61.1891|-149.906|
+896|2|F|169847.63|1993-03-09|1-URGENT|Clerk#000000187|0|inal packages eat blithely according to the warhorses. furiously quiet de|61.2191|-149.888|61.2191|-149.888|
+897|49|P|57697.44|1995-03-20|1-URGENT|Clerk#000000316|0| wake quickly against |61.2191|-149.888|61.2191|-149.888|
+898|55|F|101020.75|1993-06-03|2-HIGH|Clerk#000000611|0|. unusual pinto beans haggle quickly across |61.1101|-149.857|61.1101|-149.857|
+899|109|O|125562.09|1998-04-08|5-LOW|Clerk#000000575|0|rts engage carefully final theodolites.|61.1101|-149.857|61.1101|-149.857|
+900|46|F|120073.51|1994-10-01|4-NOT SPECIFIED|Clerk#000000060|0| fluffily express deposits nag furiousl|61.1101|-149.857|61.1101|-149.857|
+901|13|O|81826.12|1998-07-21|4-NOT SPECIFIED|Clerk#000000929|0|lyly even foxes are furious, silent requests. requests about the quickly |61.1101|-149.857|61.1101|-149.857|
+902|10|F|37348.62|1994-07-27|4-NOT SPECIFIED|Clerk#000000811|0|yly final requests over the furiously regula|61.1101|-149.857|61.1101|-149.857|
+903|11|O|109351.87|1995-07-07|4-NOT SPECIFIED|Clerk#000000793|0|e slyly about the final pl|61.1101|-149.857|61.1101|-149.857|
+928|67|F|228136.49|1995-03-02|5-LOW|Clerk#000000450|0|ithely express pinto beans. |61.1101|-149.857|61.1101|-149.857|
+929|83|F|109301.02|1992-10-02|2-HIGH|Clerk#000000160|0|its. furiously even foxes affix carefully finally silent accounts. express req|61.1101|-149.857|61.1101|-149.857|
+930|131|F|199102.23|1994-12-17|1-URGENT|Clerk#000000004|0| accounts nag slyly. ironic, ironic accounts wake blithel|61.1101|-149.857|61.1101|-149.857|
+931|103|F|117909.23|1992-12-07|1-URGENT|Clerk#000000881|0|ss packages haggle furiously express, regular deposits. even, e|61.1101|-149.857|61.1101|-149.857|
+932|41|O|40234.50|1997-05-16|2-HIGH|Clerk#000000218|0|ly express instructions boost furiously reg|61.1101|-149.857|61.1101|-149.857|
+933|97|F|71349.30|1992-08-05|4-NOT SPECIFIED|Clerk#000000752|0|ial courts wake permanently against the furiously regular ideas. unusual |61.1101|-149.857|61.1101|-149.857|
+934|52|O|17213.59|1996-07-03|1-URGENT|Clerk#000000229|0|ts integrate carefully. sly, regular deposits af|61.1101|-149.857|61.1101|-149.857|
+935|50|O|97733.87|1997-09-24|5-LOW|Clerk#000000180|0|iously final deposits cajole. blithely even packages |61.1101|-149.857|61.1101|-149.857|
+960|35|F|63537.13|1994-09-21|3-MEDIUM|Clerk#000000120|0|regular accounts. requests|61.1101|-149.857|61.1101|-149.857|
+961|56|P|158893.16|1995-06-04|4-NOT SPECIFIED|Clerk#000000720|0|ons nag furiously among the quickl|61.1101|-149.857|61.1101|-149.857|
+962|37|F|98258.73|1994-05-06|5-LOW|Clerk#000000463|0|ments nag deposits. fluffily ironic a|61.1101|-149.857|61.1101|-149.857|
+963|26|F|53287.25|1994-05-26|3-MEDIUM|Clerk#000000497|0|uses haggle carefully. slyly even dependencies after the packages ha|61.1101|-149.857|61.1101|-149.857|
+964|76|O|131146.47|1995-05-20|3-MEDIUM|Clerk#000000657|0|print blithely ironic, careful theodolit|61.1101|-149.857|61.1101|-149.857|
+965|70|P|41758.44|1995-05-15|5-LOW|Clerk#000000218|0|iously special packages. slyly pending requests are carefully |64.8591|-147.917|64.8591|-147.917|
+966|14|O|120516.93|1998-04-30|2-HIGH|Clerk#000000239|0|special deposits. furious|64.8273|-147.715|64.8273|-147.715|
+967|110|F|179287.95|1992-06-21|3-MEDIUM|Clerk#000000167|0|excuses engage quickly bold dep|64.8281|-147.715|64.8281|-147.715|
+992|55|O|133665.12|1997-11-11|3-MEDIUM|Clerk#000000875|0|ts. regular pinto beans thrash carefully sl|64.8552|-147.763|64.8552|-147.763|
+993|80|O|198238.65|1995-09-10|3-MEDIUM|Clerk#000000894|0|quickly express accounts among the furiously bol|64.8481|-147.684|64.8481|-147.684|
+994|2|F|41433.48|1994-04-20|5-LOW|Clerk#000000497|0|ole. slyly bold excuses nag caref|64.8522|-147.773|64.8522|-147.773|
+995|116|P|135157.92|1995-05-31|3-MEDIUM|Clerk#000000439|0|deas. blithely final deposits play. express accounts wake blithely caref|64.8467|-147.703|64.8467|-147.703|
+996|71|O|47447.63|1997-12-29|1-URGENT|Clerk#000000497|0|arefully final packages into the slyly final requests affix blit|64.8963|-147.662|64.8963|-147.662|
+997|109|O|27561.82|1997-05-19|2-HIGH|Clerk#000000651|0|ly express depths. furiously final requests haggle furiously. carefu|64.8372|-147.796|64.8372|-147.796|
+998|32|F|65269.38|1994-11-26|4-NOT SPECIFIED|Clerk#000000956|0|ronic dolphins. ironic, bold ideas haggle furiously furious|64.8312|-147.716|64.8312|-147.716|
+999|61|F|145249.13|1993-09-05|5-LOW|Clerk#000000464|0|pitaphs sleep. regular accounts use. f|64.811|-147.71|64.811|-147.71|
+1024|4|O|176084.63|1997-12-23|5-LOW|Clerk#000000903|0| blithely. even, express theodolites cajole slyly across|64.8971|-147.663|64.8971|-147.663|
+1025|103|F|82034.03|1995-05-05|2-HIGH|Clerk#000000376|0|ross the slyly final pa|64.85|-147.699|64.85|-147.699|
+1026|73|O|36464.76|1997-06-04|5-LOW|Clerk#000000223|0|s wake blithely. special acco|64.8389|-147.743|64.8389|-147.743|
+1027|128|F|112770.89|1992-06-03|3-MEDIUM|Clerk#000000241|0|equests cajole. slyly final pinto bean|64.781|-148|64.781|-148|
+1028|70|F|153864.67|1994-01-01|2-HIGH|Clerk#000000131|0|ts are. final, silent deposits are among the fl|64.8377|-147.718|64.8377|-147.718|
+1029|130|F|47440.91|1994-06-21|2-HIGH|Clerk#000000700|0|quests sleep. slyly even foxes wake quickly final theodolites. clo|64.8248|-147.886|64.8248|-147.886|
+1030|134|F|16346.94|1994-06-15|5-LOW|Clerk#000000422|0|ully ironic accounts sleep carefully. requests are carefully alongside of the |64.818|-147.679|64.818|-147.679|
+1031|4|F|128024.71|1994-09-01|3-MEDIUM|Clerk#000000448|0|s; ironic theodolites along the carefully ex|64.8271|-147.79|64.8271|-147.79|
+1056|28|F|38446.39|1995-02-11|1-URGENT|Clerk#000000125|0|t, even deposits hang about the slyly special i|64.8451|-147.812|64.8451|-147.812|
+1057|76|F|108107.42|1992-02-20|1-URGENT|Clerk#000000124|0|cuses dazzle carefully careful, ironic pinto beans. carefully even theod|64.8311|-147.729|64.8311|-147.729|
+1058|53|F|89359.11|1993-04-26|3-MEDIUM|Clerk#000000373|0|kly pending courts haggle. blithely regular sheaves integrate carefully fi|64.8454|-147.855|64.8454|-147.855|
+1059|127|F|198360.22|1994-02-27|1-URGENT|Clerk#000000104|0|en accounts. carefully bold packages cajole daringly special depende|64.8302|-147.744|64.8302|-147.744|
+1060|140|F|121994.04|1993-02-21|3-MEDIUM|Clerk#000000989|0|l platelets sleep quickly slyly special requests. furiously |64.8113|-147.91|64.8113|-147.91|
+1061|103|O|166947.75|1998-05-15|5-LOW|Clerk#000000576|0|uests sleep at the packages. fur|64.8271|-147.789|64.8271|-147.789|
+1062|106|O|39805.04|1997-01-15|1-URGENT|Clerk#000000152|0|eposits use blithely |64.8451|-147.698|64.8451|-147.698|
+1063|37|F|41392.31|1994-04-02|2-HIGH|Clerk#000000024|0|deposits nag quickly regular deposits. quickl|64.8586|-147.69|64.8586|-147.69|
+1088|148|F|47120.41|1992-05-21|5-LOW|Clerk#000000347|0|counts are blithely. platelets print. carefully |64.8507|-147.702|64.8507|-147.702|
+1089|49|O|103192.74|1996-05-04|4-NOT SPECIFIED|Clerk#000000226|0|ns haggle ruthlessly. even requests are quickly abov|64.8371|-147.716|64.8371|-147.716|
+1090|19|O|32929.30|1997-11-15|2-HIGH|Clerk#000000300|0| furiously regular platelets haggle along the slyly unusual foxes! |64.8449|-147.743|64.8449|-147.743|
+1091|83|O|35795.22|1996-08-27|1-URGENT|Clerk#000000549|0| even pinto beans haggle quickly alongside of the eve|64.8475|-147.706|64.8475|-147.706|
+1092|124|P|85552.21|1995-03-04|3-MEDIUM|Clerk#000000006|0|re quickly along the blithe|64.8452|-147.714|64.8452|-147.714|
+1093|101|O|79189.58|1997-07-31|4-NOT SPECIFIED|Clerk#000000159|0| after the carefully ironic requests. carefully ironic packages wake fluffil|64.8125|-147.787|64.8125|-147.787|
+1094|145|O|9006.25|1997-12-24|3-MEDIUM|Clerk#000000570|0|beans affix furiously about the pending, even deposits. finally pendi|55.3801|-131.682|55.3801|-131.682|
+1095|145|O|178491.24|1995-08-22|3-MEDIUM|Clerk#000000709|0|sly bold requests cajole carefully according to|55.3801|-131.682|55.3801|-131.682|
+1120|140|O|107958.62|1997-11-07|3-MEDIUM|Clerk#000000319|0|lly special requests. slyly pending platelets are quickly pending requ|55.3801|-131.682|55.3801|-131.682|
+1121|29|O|241837.88|1997-01-13|3-MEDIUM|Clerk#000000541|0|r escapades. deposits above the fluffily bold requests hag|55.3801|-131.682|55.3801|-131.682|
+1122|121|O|179747.47|1997-01-10|1-URGENT|Clerk#000000083|0|uffily carefully final theodolites. furiously express packages affix|55.3801|-131.682|55.3801|-131.682|
+1123|73|O|93259.93|1996-08-03|3-MEDIUM|Clerk#000000929|0|uriously pending requests. slyly regular instruction|55.3801|-131.682|55.3801|-131.682|
+1124|80|O|141858.97|1998-07-30|5-LOW|Clerk#000000326|0|regular pinto beans along the fluffily silent packages|55.3599|-131.687|55.3599|-131.687|
+1125|25|F|80438.38|1994-10-27|2-HIGH|Clerk#000000510|0|ithely final requests. i|55.4381|-131.803|55.4381|-131.803|
+1126|145|O|59982.31|1998-01-28|4-NOT SPECIFIED|Clerk#000000928|0|d slyly regular ideas: special ideas believe slyly. slyly ironic sheaves w|55.3751|-131.718|55.3751|-131.718|
+1127|58|O|103320.91|1995-09-19|4-NOT SPECIFIED|Clerk#000000397|0|usly silent, regular pinto beans. blithely express requests boos|55.3421|-131.641|55.3421|-131.641|
+1152|49|F|51775.54|1994-08-14|4-NOT SPECIFIED|Clerk#000000496|0|equests. deposits ab|55.3408|-131.64|55.3408|-131.64|
+1153|121|O|220727.97|1996-04-18|5-LOW|Clerk#000000059|0| across the pending deposi|55.2978|-131.534|55.2978|-131.534|
+1154|37|F|192417.85|1992-02-15|1-URGENT|Clerk#000000268|0|old asymptotes are special requests. blithely even deposits sleep furiously|55.3801|-131.682|55.3801|-131.682|
+1155|149|O|126902.81|1997-10-06|2-HIGH|Clerk#000000164|0|c deposits haggle among the ironic, even requests. carefully ironic sheaves n|55.3801|-131.682|55.3801|-131.682|
+1156|133|O|217682.81|1996-10-19|1-URGENT|Clerk#000000200|0| blithely ironic dolphins. furiously pendi|55.3421|-131.622|55.3421|-131.622|
+1157|97|O|85394.06|1998-01-14|4-NOT SPECIFIED|Clerk#000000207|0|out the regular excuses boost carefully against the furio|55.3801|-131.682|55.3801|-131.682|
+1158|142|O|31075.51|1996-06-30|2-HIGH|Clerk#000000549|0|integrate slyly furiously ironic deposit|55.3801|-131.682|55.3801|-131.682|
+1159|70|F|55553.68|1992-09-18|3-MEDIUM|Clerk#000000992|0|ts may sleep. requests according to the|55.3801|-131.682|55.3801|-131.682|
+1184|89|O|39700.29|1997-10-26|5-LOW|Clerk#000000777|0|iously even packages haggle fluffily care|55.3267|-131.523|55.3267|-131.523|
+1185|74|F|47033.21|1992-08-24|5-LOW|Clerk#000000344|0| even escapades are. package|55.3522|-131.685|55.3522|-131.685|
+1186|59|O|82026.18|1996-08-15|4-NOT SPECIFIED|Clerk#000000798|0|ingly regular pinto beans: instructi|55.5351|-133.014|55.5351|-133.014|
+1187|134|F|85948.02|1992-11-20|3-MEDIUM|Clerk#000000047|0|s after the furiously final deposits boost slyly under the|55.5351|-133.014|55.5351|-133.014|
+1188|20|O|54655.07|1996-04-11|2-HIGH|Clerk#000000256|0|ully ironic deposits. slyl|55.5351|-133.014|55.5351|-133.014|
+1189|46|F|71017.99|1994-04-09|1-URGENT|Clerk#000000243|0|f the even accounts. courts print blithely ironic accounts. sile|55.5351|-133.014|55.5351|-133.014|
+1190|13|O|31043.39|1997-03-16|5-LOW|Clerk#000000575|0|ccounts above the foxes integrate carefully after the |55.5351|-133.014|55.5351|-133.014|
+1191|112|O|28623.04|1995-11-07|3-MEDIUM|Clerk#000000011|0|uests nag furiously. carefully even requests|55.4691|-132.855|55.4691|-132.855|
+1216|122|F|68056.57|1992-12-07|5-LOW|Clerk#000000918|0|nal foxes around the e|55.5511|-133.081|55.5511|-133.081|
+1217|7|F|40982.08|1992-04-26|4-NOT SPECIFIED|Clerk#000000538|0| foxes nag quickly. ironic excuses nod. blithely pending|55.5351|-133.014|55.5351|-133.014|
+1218|10|F|99834.47|1994-06-20|4-NOT SPECIFIED|Clerk#000000994|0|s cajole. special, silent deposits about the theo|55.5531|-133.097|55.5531|-133.097|
+1219|28|O|10163.56|1995-10-05|3-MEDIUM|Clerk#000000800|0|od carefully. slyly final dependencies across the even fray|55.5351|-133.014|55.5351|-133.014|
+1220|49|O|122157.14|1996-08-29|1-URGENT|Clerk#000000712|0|inal theodolites wake. fluffily ironic asymptotes cajol|55.4726|-131.793|55.4726|-131.793|
+1221|14|F|117397.16|1992-04-19|4-NOT SPECIFIED|Clerk#000000852|0| detect against the silent, even deposits. carefully ironic|55.3801|-131.682|55.3801|-131.682|
+1222|10|F|47623.94|1993-02-05|3-MEDIUM|Clerk#000000811|0|theodolites use quickly even accounts. carefully final asympto|55.3801|-131.682|55.3801|-131.682|
+1223|10|O|26714.67|1996-05-25|4-NOT SPECIFIED|Clerk#000000238|0|posits was blithely fr|55.3801|-131.682|55.3801|-131.682|
+1248|49|F|210713.88|1992-01-02|1-URGENT|Clerk#000000890|0|t the carefully regular dugouts. s|61.5745|-149.562|61.5745|-149.562|
+1249|149|F|45889.09|1994-01-05|1-URGENT|Clerk#000000095|0|al ideas sleep above the pending pin|61.7321|-150.12|61.7321|-150.12|
+1250|37|F|12907.62|1992-09-29|4-NOT SPECIFIED|Clerk#000000652|0|ts after the fluffily pending instructions use slyly about the s|61.5421|-149.419|61.5421|-149.419|
+1251|38|O|109536.55|1997-10-30|1-URGENT|Clerk#000000276|0|, brave sauternes. deposits boost fluffily.|61.5722|-149.702|61.5722|-149.702|
+1252|149|O|93403.05|1997-08-04|5-LOW|Clerk#000000348|0|ng the slyly regular excuses. special courts nag furiously blithely e|61.5743|-149.405|61.5743|-149.405|
+1253|115|F|92730.74|1993-01-26|1-URGENT|Clerk#000000775|0| requests sleep furiously even foxes. ruthless packag|61.578|-149.441|61.578|-149.441|
+1254|70|O|94649.25|1995-12-22|1-URGENT|Clerk#000000607|0| pinto beans. carefully regular request|61.5826|-149.427|61.5826|-149.427|
+1255|122|F|62518.31|1994-05-30|4-NOT SPECIFIED|Clerk#000000798|0|ct slyly regular accounts. quick|61.5586|-149.351|61.5586|-149.351|
+1280|97|F|91664.85|1993-01-11|5-LOW|Clerk#000000160|0|posits thrash quickly after the theodolites. furiously iro|61.5844|-149.442|61.5844|-149.442|
+1281|62|F|165454.51|1994-12-11|1-URGENT|Clerk#000000430|0|counts. carefully pending accounts eat |61.5817|-149.472|61.5817|-149.472|
+1282|116|F|61297.42|1992-02-29|4-NOT SPECIFIED|Clerk#000000168|0|he quickly special packages. furiously final re|61.6141|-149.457|61.6141|-149.457|
+1283|118|O|202623.92|1996-08-30|4-NOT SPECIFIED|Clerk#000000260|0| pinto beans boost slyly ac|61.5761|-149.602|61.5761|-149.602|
+1284|134|O|106122.38|1996-01-07|2-HIGH|Clerk#000000492|0|s. blithely silent deposits s|61.6141|-149.457|61.6141|-149.457|
+1285|11|F|139124.72|1992-06-01|1-URGENT|Clerk#000000423|0|cial deposits cajole after the ironic requests. p|61.58|-149.434|61.58|-149.434|
+1286|109|F|207291.83|1993-05-14|4-NOT SPECIFIED|Clerk#000000939|0| deposits use carefully from the excuses. slyly bold p|61.6002|-149.429|61.6002|-149.429|
+1287|19|F|131432.42|1994-07-05|2-HIGH|Clerk#000000288|0|ly ironic dolphins integrate furiously among the final packages. st|61.569|-149.347|61.569|-149.347|
+1312|112|F|58111.00|1994-05-19|3-MEDIUM|Clerk#000000538|0|n, express accounts across the ironic|61.5812|-149.448|61.5812|-149.448|
+1313|148|F|46598.65|1994-09-13|1-URGENT|Clerk#000000774|0|ld accounts. regular deposits cajole. ironically pending theodolites use car|61.6141|-149.457|61.6141|-149.457|
+1314|143|F|56207.66|1994-05-13|3-MEDIUM|Clerk#000000485|0|ickly blithe packages nod ideas. furiously bold braids boost around the car|61.6141|-149.457|61.6141|-149.457|
+1315|22|O|121935.23|1998-03-22|5-LOW|Clerk#000000840|0|final theodolites alongside of the carefu|61.6141|-149.457|61.6141|-149.457|
+1316|16|F|163746.47|1993-12-03|1-URGENT|Clerk#000000857|0|ully bold theodolites? pending, bold pin|61.5969|-149.367|61.5969|-149.367|
+1317|100|P|139714.71|1995-05-19|2-HIGH|Clerk#000000373|0|sts. furiously special deposits lose fur|61.58|-149.4|61.58|-149.4|
+1318|128|O|81663.65|1998-06-27|3-MEDIUM|Clerk#000000581|0|s hang bold requests. pending, re|61.5848|-149.445|61.5848|-149.445|
+1319|32|O|31103.83|1996-09-27|2-HIGH|Clerk#000000257|0|y across the ruthlessly ironic accounts. unusu|61.5811|-149.444|61.5811|-149.444|
+1344|17|F|43809.37|1992-04-16|5-LOW|Clerk#000000178|0|omise close, silent requests. pending theodolites boost pending |61.5733|-149.389|61.5733|-149.389|
+1345|95|F|111207.93|1992-10-28|5-LOW|Clerk#000000447|0| regular tithes. quickly fluffy de|61.6141|-149.457|61.6141|-149.457|
+1346|76|F|171975.62|1992-06-18|2-HIGH|Clerk#000000374|0|ges sleep quickly-- even pint|61.5952|-149.436|61.5952|-149.436|
+1347|41|O|173444.60|1997-06-20|5-LOW|Clerk#000000977|0|he furiously even foxes use carefully express req|61.5421|-149.419|61.5421|-149.419|
+1348|19|O|94135.77|1998-04-18|5-LOW|Clerk#000000206|0|tly. quickly even deposi|61.5783|-149.362|61.5783|-149.362|
+1349|64|O|46376.09|1997-10-26|1-URGENT|Clerk#000000543|0|yly! blithely special theodolites cajole. unusual, reg|61.7321|-150.12|61.7321|-150.12|
+1350|52|F|49305.98|1993-08-24|1-URGENT|Clerk#000000635|0|iously about the blithely special a|61.5691|-149.328|61.5691|-149.328|
+1351|106|O|24637.96|1998-04-20|1-URGENT|Clerk#000000012|0| cajole. regular, special re|61.6141|-149.457|61.6141|-149.457|
+1376|47|O|23984.88|1997-05-04|4-NOT SPECIFIED|Clerk#000000730|0|der furiously final, final frets. carefull|61.5819|-149.3|61.5819|-149.3|
+1377|20|O|108334.30|1998-04-24|4-NOT SPECIFIED|Clerk#000000625|0|lly across the blithely express accounts. ironic excuses promise carefully de|61.6431|-149.289|61.6431|-149.289|
+1378|20|O|118495.12|1996-03-09|4-NOT SPECIFIED|Clerk#000000705|0| furiously even tithes cajole slyly among the quick|61.6431|-149.292|61.6431|-149.292|
+1379|65|O|84627.76|1998-05-25|5-LOW|Clerk#000000861|0|y deposits are caref|61.6228|-149.313|61.6228|-149.313|
+1380|137|O|94969.41|1996-07-07|3-MEDIUM|Clerk#000000969|0|inal deposits wake slyly daringly even requests. bold, even foxe|61.2125|-149.894|61.2125|-149.894|
+1381|127|O|58212.22|1998-05-25|3-MEDIUM|Clerk#000000107|0|even requests breach after the bold, ironic instructions. slyly even|61.1879|-149.886|61.1879|-149.886|
+1382|133|F|173522.71|1993-08-17|5-LOW|Clerk#000000241|0|fully final packages sl|61.1594|-149.835|61.1594|-149.835|
+1383|121|F|34797.72|1993-04-27|2-HIGH|Clerk#000000785|0|ts. express requests sleep blithel|61.2123|-149.854|61.2123|-149.854|
+1408|55|O|183965.61|1997-12-26|4-NOT SPECIFIED|Clerk#000000942|0|t the quickly final asymptotes. unusual|61.1951|-149.945|61.1951|-149.945|
+1409|143|F|72440.52|1992-12-31|4-NOT SPECIFIED|Clerk#000000065|0|ructions. furiously unusual excuses are regular, unusual theodolites. fin|61.2138|-149.856|61.2138|-149.856|
+1410|113|O|114879.19|1997-04-12|5-LOW|Clerk#000000123|0|iously along the bravely regular dolphins. pinto beans cajole furiously sp|61.1255|-149.864|61.1255|-149.864|
+1411|95|F|164462.61|1994-12-21|2-HIGH|Clerk#000000566|0|s. furiously special excuses across the pending pinto beans haggle sp|61.2066|-149.808|61.2066|-149.808|
+1412|53|F|78676.54|1993-03-13|4-NOT SPECIFIED|Clerk#000000083|0|uffily daring theodolit|61.2138|-149.896|61.2138|-149.896|
+1413|91|O|75733.58|1997-06-14|3-MEDIUM|Clerk#000000342|0|, ironic instructions. carefully even packages dazzle|61.2161|-149.876|61.2161|-149.876|
+1414|77|O|38057.81|1995-08-16|1-URGENT|Clerk#000000883|0|ccounts. ironic foxes haggle car|61.1594|-149.888|61.1594|-149.888|
+1415|79|F|24654.79|1994-05-29|4-NOT SPECIFIED|Clerk#000000601|0|rays. blithely final ideas affix quickl|61.1806|-149.775|61.1806|-149.775|
+1440|98|O|50201.16|1995-08-10|5-LOW|Clerk#000000956|0| pending requests. closely s|61.1101|-149.857|61.1101|-149.857|
+1441|122|O|156477.94|1997-03-06|4-NOT SPECIFIED|Clerk#000000156|0|ter the excuses. ironic dependencies m|61.1541|-149.958|61.1541|-149.958|
+1442|112|F|7108.12|1994-07-05|4-NOT SPECIFIED|Clerk#000000935|0|nal pinto beans. slyly ironic ideas cajol|61.1268|-149.947|61.1268|-149.947|
+1443|44|O|44672.03|1996-12-16|5-LOW|Clerk#000000185|0|x blithely against the carefully final somas. even asymptotes are. quickly spe|61.0931|-149.785|61.0931|-149.785|
+1444|134|F|207907.60|1994-12-06|3-MEDIUM|Clerk#000000783|0|ove the bold accounts cajole fluffily about|61.1901|-149.892|61.1901|-149.892|
+1445|115|F|154653.32|1995-01-10|3-MEDIUM|Clerk#000000211|0|even packages wake fluffily |61.2183|-149.889|61.2183|-149.889|
+1446|41|O|27663.16|1998-02-16|5-LOW|Clerk#000000274|0|lly regular notornis above the requests sleep final accounts! |61.2164|-149.882|61.2164|-149.882|
+1447|91|F|108171.38|1992-10-15|2-HIGH|Clerk#000000880|0|inly against the blithely pending excuses. regular, pe|61.2161|-149.876|61.2161|-149.876|
+1472|149|O|65331.05|1996-10-06|5-LOW|Clerk#000000303|0|y special dolphins around the final dependencies wake quick|61.219|-149.792|61.219|-149.792|
+1473|94|O|80624.38|1997-03-17|3-MEDIUM|Clerk#000000960|0|furiously close accoun|61.2188|-149.892|61.2188|-149.892|
+1474|70|F|51697.18|1995-01-09|1-URGENT|Clerk#000000438|0|detect quickly above the carefully even |61.2143|-149.837|61.2143|-149.837|
+1475|5|O|185496.66|1997-11-12|2-HIGH|Clerk#000000972|0|cally final packages boost. blithely ironic packa|61.1608|-149.835|61.1608|-149.835|
+1476|145|O|18795.62|1996-06-27|2-HIGH|Clerk#000000673|0|ding accounts hinder alongside of the quickly pending requests. fluf|61.1886|-149.944|61.1886|-149.944|
+1477|76|O|231831.35|1997-08-24|5-LOW|Clerk#000000612|0|ly bold foxes. final ideas would cajo|61.1201|-149.89|61.1201|-149.89|
+1478|50|O|20791.50|1997-08-03|2-HIGH|Clerk#000000827|0|lessly. carefully express|61.1201|-149.89|61.1201|-149.89|
+1479|16|O|31471.04|1995-12-16|4-NOT SPECIFIED|Clerk#000000697|0|he furiously even foxes. thinly bold deposits|61.1585|-149.872|61.1585|-149.872|
+1504|2|F|89399.40|1992-08-28|3-MEDIUM|Clerk#000000381|0|, brave deposits. bold de|61.195|-149.892|61.195|-149.892|
+1505|37|F|55892.35|1992-08-21|2-HIGH|Clerk#000000544|0|s. slyly ironic packages cajole. carefully regular packages haggle |61.0895|-149.694|61.0895|-149.694|
+1506|148|F|195844.84|1992-09-21|3-MEDIUM|Clerk#000000620|0| dependencies. accounts affix blithely slowly unusual deposits. slyly regular |61.2201|-149.831|61.2201|-149.831|
+1507|121|F|96166.92|1993-10-14|3-MEDIUM|Clerk#000000305|0|stealthy, ironic de|61.1663|-149.867|61.1663|-149.867|
+1508|103|O|151282.65|1998-04-10|5-LOW|Clerk#000000117|0| after the furiously regular pinto beans hang slyly quickly ironi|61.2138|-149.906|61.2138|-149.906|
+1509|64|F|180455.98|1993-07-08|5-LOW|Clerk#000000770|0|the regular ideas. regul|61.2193|-149.902|61.2193|-149.902|
+1510|53|O|154590.05|1996-09-17|5-LOW|Clerk#000000128|0|ld carefully. furiously final asymptotes haggle furiously|61.1201|-149.89|61.1201|-149.89|
+1511|79|O|59651.38|1996-12-22|4-NOT SPECIFIED|Clerk#000000386|0|ts above the depend|61.1601|-149.984|61.1601|-149.984|
+1536|94|O|5184.26|1997-01-26|3-MEDIUM|Clerk#000000117|0|ges are! furiously final deposits cajole iron|61.1101|-149.857|61.1101|-149.857|
+1537|109|F|108317.51|1992-02-15|4-NOT SPECIFIED|Clerk#000000862|0|g to the even deposits. ironic, final packages |61.1101|-149.857|61.1101|-149.857|
+1538|29|O|179554.41|1995-06-18|4-NOT SPECIFIED|Clerk#000000258|0| instructions. regular theod|61.1101|-149.857|61.1101|-149.857|
+1539|112|F|39612.63|1995-03-10|5-LOW|Clerk#000000840|0|nstructions boost pa|61.1101|-149.857|61.1101|-149.857|
+1540|16|F|128014.15|1992-08-05|2-HIGH|Clerk#000000927|0|r ideas hinder blithe|61.1101|-149.857|61.1101|-149.857|
+1541|94|P|47286.32|1995-05-18|1-URGENT|Clerk#000000906|0|y. slyly ironic warhorses around the furiously regul|61.1101|-149.857|61.1101|-149.857|
+1542|143|F|132972.24|1993-09-15|3-MEDIUM|Clerk#000000435|0|t the furiously close deposits do was f|61.1101|-149.857|61.1101|-149.857|
+1543|52|O|139047.22|1997-02-20|1-URGENT|Clerk#000000398|0|unts. furiously pend|61.1101|-149.857|61.1101|-149.857|
+1568|17|O|76119.72|1997-01-30|4-NOT SPECIFIED|Clerk#000000554|0|d notornis. carefully |61.1101|-149.857|61.1101|-149.857|
+1569|104|O|87803.55|1998-04-02|5-LOW|Clerk#000000786|0|orbits. fluffily even decoys serve blithely. furiously furious realms nag acro|61.1101|-149.857|61.1101|-149.857|
+1570|124|O|35589.57|1998-03-16|1-URGENT|Clerk#000000745|0|pinto beans haggle furiousl|61.1101|-149.857|61.1101|-149.857|
+1571|103|F|151404.78|1992-12-05|2-HIGH|Clerk#000000565|0|ously furiously bold warthogs. slyly ironic instructions are quickly a|61.1101|-149.857|61.1101|-149.857|
+1572|11|O|47232.79|1996-02-24|2-HIGH|Clerk#000000994|0|fluffily ironic accounts haggle blithely final platelets! slyly regular foxes|61.1101|-149.857|61.1101|-149.857|
+1573|148|F|86918.57|1992-12-28|2-HIGH|Clerk#000000940|0|ess, ironic deposits use along the carefu|61.1101|-149.857|61.1101|-149.857|
+1574|134|O|179923.54|1996-12-12|3-MEDIUM|Clerk#000000809|0| ideas hinder after the carefully unusual |61.1101|-149.857|61.1101|-149.857|
+1575|145|O|197031.52|1995-09-13|3-MEDIUM|Clerk#000000497|0|. furiously regular dep|61.1101|-149.857|61.1101|-149.857|
+1600|94|F|130515.61|1993-03-03|3-MEDIUM|Clerk#000000627|0|tions cajole quietly above the regular, silent requests. slyly fin|61.1101|-149.857|61.1101|-149.857|
+1601|53|F|73962.95|1994-08-27|5-LOW|Clerk#000000469|0|ent deposits are ca|61.1101|-149.857|61.1101|-149.857|
+1602|1|F|4225.26|1993-08-05|5-LOW|Clerk#000000660|0|deposits. busily silent instructions haggle furiously. fin|61.1101|-149.857|61.1101|-149.857|
+1603|2|F|29305.47|1993-07-31|4-NOT SPECIFIED|Clerk#000000869|0|s. slyly silent deposits boo|61.1101|-149.857|61.1101|-149.857|
+1604|113|F|107139.29|1993-07-17|5-LOW|Clerk#000000512|0|lithely silent waters. blithely unusual packages alongside |61.1101|-149.857|61.1101|-149.857|
+1605|58|O|130687.64|1998-04-24|4-NOT SPECIFIED|Clerk#000000616|0|sleep furiously? ruthless, even pinto beans |61.1101|-149.857|61.1101|-149.857|
+1606|53|O|115877.40|1997-04-17|4-NOT SPECIFIED|Clerk#000000550|0|r requests. quickly even platelets breach before the ironically|61.1101|-149.857|61.1101|-149.857|
+1607|149|O|166335.03|1995-12-16|2-HIGH|Clerk#000000498|0| bold, pending foxes haggle. slyly silent |61.1101|-149.857|61.1101|-149.857|
+1632|67|O|183286.33|1997-01-08|3-MEDIUM|Clerk#000000351|0|onic requests are accounts. bold a|61.1101|-149.857|61.1101|-149.857|
+1633|16|O|52359.51|1995-10-14|2-HIGH|Clerk#000000666|0|y silent accounts sl|61.1101|-149.857|61.1101|-149.857|
+1634|70|O|145898.47|1996-09-10|1-URGENT|Clerk#000000360|0|arefully blithely ironic requests. slyly unusual instructions alongside|61.1101|-149.857|61.1101|-149.857|
+1635|4|O|70232.26|1997-02-13|3-MEDIUM|Clerk#000000958|0|s. slyly ironic requests affix slyly |61.1101|-149.857|61.1101|-149.857|
+1636|79|O|172021.87|1997-06-17|3-MEDIUM|Clerk#000000457|0|ding requests. slyly ironic courts wake quickl|61.1101|-149.857|61.1101|-149.857|
+1637|73|F|180912.15|1995-02-08|4-NOT SPECIFIED|Clerk#000000189|0| final accounts. blithely silent ideas cajole bravely. carefully express |61.1101|-149.857|61.1101|-149.857|
+1638|139|O|172436.30|1997-08-13|2-HIGH|Clerk#000000643|0|he fluffily regular asymp|61.1101|-149.857|61.1101|-149.857|
+1639|5|O|104166.56|1995-08-20|4-NOT SPECIFIED|Clerk#000000939|0|haggle furiously. final requests detect furious|61.1101|-149.857|61.1101|-149.857|
+1664|64|O|178060.22|1996-03-03|1-URGENT|Clerk#000000090|0|y quickly even asymptotes. furiously regular packages haggle quickly fin|61.1101|-149.857|61.1101|-149.857|
+1665|76|F|4819.91|1994-05-08|2-HIGH|Clerk#000000920|0|ly regular packages are fluffily even ideas. fluffily final|61.1101|-149.857|61.1101|-149.857|
+1666|95|O|128367.97|1995-10-18|1-URGENT|Clerk#000000849|0|ffily pending dependencies wake fluffily. pending, final accounts |61.1101|-149.857|61.1101|-149.857|
+1667|5|O|125030.37|1997-10-10|2-HIGH|Clerk#000000103|0|e accounts. slyly express accounts must are a|64.8459|-147.759|64.8459|-147.759|
+1668|142|O|137576.19|1997-07-12|4-NOT SPECIFIED|Clerk#000000148|0|eodolites. carefully dogged dolphins haggle q|64.8426|-147.725|64.8426|-147.725|
+1669|2|O|24362.39|1997-06-09|3-MEDIUM|Clerk#000000663|0|er ironic requests detect furiously blithely sp|64.9401|-147.402|64.9401|-147.402|
+1670|25|O|89999.72|1997-05-24|2-HIGH|Clerk#000000320|0|unusual dependencies. furiously special platelets main|64.9401|-147.402|64.9401|-147.402|
+1671|35|O|104391.11|1996-07-27|4-NOT SPECIFIED|Clerk#000000275|0|ly. slyly pending requests was above the |64.8331|-147.647|64.8331|-147.647|
+1696|4|O|102665.03|1998-01-08|4-NOT SPECIFIED|Clerk#000000041|0|bravely bold accounts above the quickly bold|64.8371|-147.716|64.8371|-147.716|
+1697|76|O|122621.31|1996-10-07|1-URGENT|Clerk#000000815|0|o x-ray blithely. pl|64.8574|-147.759|64.8574|-147.759|
+1698|40|O|141118.87|1997-04-23|2-HIGH|Clerk#000000432|0|slyly. carefully express deposit|64.836|-147.727|64.836|-147.727|
+1699|85|F|66408.29|1993-12-30|1-URGENT|Clerk#000000125|0|jole blithely. furiously un|64.8132|-147.76|64.8132|-147.76|
+1700|65|O|89143.36|1996-06-15|3-MEDIUM|Clerk#000000328|0|ely final dolphins wake sometimes above the quietly regular deposits. fur|64.8451|-147.96|64.8451|-147.96|
+1701|130|F|72835.95|1992-05-19|2-HIGH|Clerk#000000395|0|furiously. regular, close theodoli|64.8891|-147.851|64.8891|-147.851|
+1702|67|P|194119.31|1995-05-07|2-HIGH|Clerk#000000300|0|around the carefully final deposits cajole carefully according to the b|64.8151|-147.707|64.8151|-147.707|
+1703|134|F|121220.59|1993-01-28|3-MEDIUM|Clerk#000000463|0| pinto beans poach. bold courts boost. regular, express deposits at|64.8363|-147.803|64.8363|-147.803|
+1728|64|O|131604.34|1996-05-22|2-HIGH|Clerk#000000711|0|beans. slyly regular instructions sleep! slyly final packages|64.8298|-147.611|64.8298|-147.611|
+1729|133|F|12137.76|1992-05-19|2-HIGH|Clerk#000000158|0|pending foxes wake. accounts|64.8989|-147.701|64.8989|-147.701|
+1730|124|O|150886.49|1998-07-24|5-LOW|Clerk#000000794|0| fluffily pending deposits serve. furiously even requests wake furiou|64.8371|-147.716|64.8371|-147.716|
+1731|128|O|190490.78|1996-01-06|1-URGENT|Clerk#000000268|0|lithely regular, final instructions. ironic, express packages are above|64.8147|-147.706|64.8147|-147.706|
+1732|146|F|179854.51|1993-11-29|5-LOW|Clerk#000000903|0|inal requests integrate dolph|64.8451|-147.812|64.8451|-147.812|
+1733|148|O|165489.52|1996-05-12|2-HIGH|Clerk#000000789|0|e carefully according to the accounts. furiously pending instructions sleep|64.8386|-147.788|64.8386|-147.788|
+1734|7|F|44002.53|1994-06-11|2-HIGH|Clerk#000000722|0| final ideas haggle. blithely quick foxes sleep busily bold ideas. i|64.8372|-147.768|64.8372|-147.768|
+1735|22|F|98541.95|1992-12-27|1-URGENT|Clerk#000000458|0|ully idle requests wake qu|64.8151|-147.707|64.8151|-147.707|
+1760|115|O|82151.12|1996-05-17|5-LOW|Clerk#000000917|0| deposits. busily regular deposits wake blithely along the furiously even re|64.843|-147.722|64.843|-147.722|
+1761|106|F|211925.95|1993-12-24|2-HIGH|Clerk#000000817|0|efully slyly bold frets. packages boost b|64.8426|-147.725|64.8426|-147.725|
+1762|77|F|202227.17|1994-08-20|4-NOT SPECIFIED|Clerk#000000653|0|ly ironic packages. furi|64.8615|-147.723|64.8615|-147.723|
+1763|121|O|140685.01|1996-10-29|2-HIGH|Clerk#000000321|0|es. bold dependencies haggle furiously along |64.8694|-147.067|64.8694|-147.067|
+1764|29|F|47384.71|1992-03-25|1-URGENT|Clerk#000000182|0|. slyly final packages integrate carefully acro|64.8404|-147.724|64.8404|-147.724|
+1765|73|O|36551.43|1995-12-03|4-NOT SPECIFIED|Clerk#000000490|0| regular excuses wake slyly|64.9686|-147.577|64.9686|-147.577|
+1766|139|O|41032.81|1996-10-12|2-HIGH|Clerk#000000983|0|unusual deposits affix quickly beyond the carefully s|64.8497|-147.732|64.8497|-147.732|
+1767|25|P|136582.60|1995-03-14|2-HIGH|Clerk#000000327|0|eposits use carefully carefully regular platelets. quickly regular packages al|64.8861|-147.587|64.8861|-147.587|
+1792|49|F|107919.86|1993-11-09|5-LOW|Clerk#000000102|0|ructions haggle along the pending packages. carefully speci|64.8508|-147.703|64.8508|-147.703|
+1793|19|F|82504.56|1992-07-12|4-NOT SPECIFIED|Clerk#000000291|0|regular packages cajole. blithely special packages according to the final d|64.841|-147.72|64.841|-147.72|
+1794|140|O|179462.21|1997-09-28|1-URGENT|Clerk#000000686|0|ally silent pinto beans. regular package|64.8375|-147.721|64.8375|-147.721|
+1795|94|F|146849.33|1994-03-19|2-HIGH|Clerk#000000815|0| quickly final packages! blithely dogged accounts c|64.849|-147.813|64.849|-147.813|
+1796|47|F|33755.47|1992-11-21|2-HIGH|Clerk#000000245|0|eans use furiously around th|55.3801|-131.682|55.3801|-131.682|
+1797|125|O|51494.47|1996-05-07|3-MEDIUM|Clerk#000000508|0|quiet platelets haggle since the quickly ironic instructi|55.3801|-131.682|55.3801|-131.682|
+1798|52|O|46393.97|1997-07-28|1-URGENT|Clerk#000000741|0|al foxes are blithe|55.3603|-131.702|55.3603|-131.702|
+1799|61|F|46815.93|1994-03-07|4-NOT SPECIFIED|Clerk#000000339|0|ns sleep furiously final waters. blithely regular instructions h|55.7511|-132.865|55.7511|-132.865|
+1824|49|F|81351.53|1994-05-05|1-URGENT|Clerk#000000972|0|e blithely fluffily|55.7511|-132.865|55.7511|-132.865|
+1825|148|F|150582.77|1993-12-05|3-MEDIUM|Clerk#000000345|0|ironic, final accou|60.3311|-151.284|60.3311|-151.284|
+1826|82|F|124719.97|1992-04-16|4-NOT SPECIFIED|Clerk#000000718|0|the even asymptotes dazzle fluffily slyly regular asymptotes. final, unu|60.3311|-151.284|60.3311|-151.284|
+1827|106|O|210113.88|1996-06-22|4-NOT SPECIFIED|Clerk#000000369|0|luffily even requests haggle sly|60.3311|-151.284|60.3311|-151.284|
+1828|32|F|137369.50|1994-04-18|3-MEDIUM|Clerk#000000840|0|y quickly bold packag|60.4341|-151.283|60.4341|-151.283|
+1829|112|F|127532.20|1994-05-08|2-HIGH|Clerk#000000537|0| accounts wake above the furiously unusual requests. pending package|60.3311|-151.284|60.3311|-151.284|
+1830|133|F|85122.24|1995-02-23|1-URGENT|Clerk#000000045|0|according to the even,|60.3311|-151.284|60.3311|-151.284|
+1831|71|F|58032.77|1993-12-02|1-URGENT|Clerk#000000854|0| accounts. carefully even accounts boost furiously. regular ideas engage. |60.3311|-151.284|60.3311|-151.284|
+1856|106|F|189361.42|1992-03-20|4-NOT SPECIFIED|Clerk#000000952|0|. special pinto beans run acr|60.3311|-151.284|60.3311|-151.284|
+1857|133|F|102793.59|1993-01-13|2-HIGH|Clerk#000000083|0|hely final ideas slee|60.3311|-151.284|60.3311|-151.284|
+1858|143|O|30457.91|1997-12-13|1-URGENT|Clerk#000000389|0|thely. slyly final deposits sleep|60.4311|-151.286|60.4311|-151.286|
+1859|61|O|105094.09|1997-04-11|4-NOT SPECIFIED|Clerk#000000949|0| the foxes. bravely special excuses nag carefully special r|60.3311|-151.284|60.3311|-151.284|
+1860|10|O|9103.40|1996-04-04|3-MEDIUM|Clerk#000000556|0|osits. quickly bold deposits according to |60.3311|-151.284|60.3311|-151.284|
+1861|70|F|95063.41|1994-01-03|3-MEDIUM|Clerk#000000847|0|r the fluffily close sauternes. furio|60.3311|-151.284|60.3311|-151.284|
+1862|34|O|97981.06|1998-02-24|5-LOW|Clerk#000000348|0|ts snooze ironically abou|60.3311|-151.284|60.3311|-151.284|
+1863|74|F|96359.65|1993-09-23|4-NOT SPECIFIED|Clerk#000000658|0|old sentiments. careful, |60.3191|-151.296|60.3191|-151.296|
+1888|121|F|224724.11|1993-10-31|4-NOT SPECIFIED|Clerk#000000659|0|olites. pinto beans cajole. regular deposits affix. slyly regular|60.3311|-151.284|60.3311|-151.284|
+1889|25|O|96431.77|1997-03-16|1-URGENT|Clerk#000000854|0|p around the regular notornis. unusual deposits|60.3311|-151.284|60.3311|-151.284|
+1890|10|O|202364.58|1996-12-18|4-NOT SPECIFIED|Clerk#000000627|0|romise final, regular deposits. regular fox|60.3311|-151.284|60.3311|-151.284|
+1891|61|F|76848.96|1994-12-15|5-LOW|Clerk#000000495|0|unusual foxes sleep regular deposits. requests wake special pac|60.5563|-151.241|60.5563|-151.241|
+1892|25|F|133273.64|1994-03-26|5-LOW|Clerk#000000733|0|sts. slyly regular dependencies use slyly. ironic, spec|60.6331|-151.163|60.6331|-151.163|
+1893|125|O|116792.13|1997-10-30|2-HIGH|Clerk#000000111|0|olites. silent, special deposits eat slyly quickly express packages; hockey p|60.6331|-151.163|60.6331|-151.163|
+1894|76|F|44387.23|1992-03-30|1-URGENT|Clerk#000000626|0|e furiously. furiously even accounts are slyly final accounts. closely speci|60.6331|-151.163|60.6331|-151.163|
+1895|7|F|44429.81|1994-05-30|3-MEDIUM|Clerk#000000878|0|ress accounts. bold accounts cajole. slyly final pinto beans poach regul|60.6331|-151.163|60.6331|-151.163|
+1920|110|O|119605.91|1998-06-24|5-LOW|Clerk#000000018|0|hely; furiously regular excuses|60.5551|-151.245|60.5551|-151.245|
+1921|88|F|57584.12|1994-01-18|3-MEDIUM|Clerk#000000293|0|counts. slyly quiet requests along the ruthlessly regular accounts are |60.6331|-151.163|60.6331|-151.163|
+1922|56|O|11575.77|1996-07-13|3-MEDIUM|Clerk#000000984|0|side of the blithely final re|60.5506|-151.141|60.5506|-151.141|
+1923|136|O|171128.10|1997-07-07|1-URGENT|Clerk#000000471|0| express dolphins. |60.5681|-151.281|60.5681|-151.281|
+1924|76|O|169756.19|1996-09-07|4-NOT SPECIFIED|Clerk#000000823|0| of the ironic accounts. instructions near the final instr|60.5465|-151.147|60.5465|-151.147|
+1925|17|F|146382.71|1992-03-05|1-URGENT|Clerk#000000986|0|e slyly regular deposits. furiously |60.6331|-151.163|60.6331|-151.163|
+1926|94|O|100035.03|1996-01-31|2-HIGH|Clerk#000000568|0|cajole. even warhorses sleep carefully. |60.5578|-151.116|60.5578|-151.116|
+1927|140|O|23327.88|1995-09-30|3-MEDIUM|Clerk#000000616|0|riously special packages. permanent pearls wake furiously. even packages alo|61.6182|-149.385|61.6182|-149.385|
+1952|67|F|12896.25|1994-03-16|2-HIGH|Clerk#000000254|0| silent accounts boost |61.6141|-149.457|61.6141|-149.457|
+1953|149|F|57213.18|1993-11-30|3-MEDIUM|Clerk#000000891|0| fluffily along the quickly even packages. |61.5765|-149.407|61.5765|-149.407|
+1954|56|O|158853.63|1997-05-31|4-NOT SPECIFIED|Clerk#000000104|0| unusual excuses cajole according to the blithely regular theodolites.|61.6091|-149.77|61.6091|-149.77|
+1955|13|F|103085.13|1992-04-20|1-URGENT|Clerk#000000792|0|ly special ideas. sometimes final |61.5821|-149.438|61.5821|-149.438|
+1956|127|F|88704.26|1992-09-20|4-NOT SPECIFIED|Clerk#000000600|0|ironic ideas are silent ideas. furiously final deposits sleep slyly carefu|61.6183|-149.373|61.6183|-149.373|
+1957|31|O|77482.87|1998-07-21|2-HIGH|Clerk#000000639|0|nding excuses about the |61.6131|-149.403|61.6131|-149.403|
+1958|53|O|176294.34|1995-09-22|5-LOW|Clerk#000000343|0| haggle blithely. flu|61.6352|-149.265|61.6352|-149.265|
+1959|43|O|62277.18|1997-01-13|4-NOT SPECIFIED|Clerk#000000631|0| cajole about the blithely express requests. even excuses mold bl|61.5751|-149.645|61.5751|-149.645|
+1984|52|O|79230.47|1998-04-01|1-URGENT|Clerk#000000416|0| slyly special instructions. unusual foxes use packages. carefully regular req|61.6168|-149.374|61.6168|-149.374|
+1985|7|F|171522.54|1994-09-02|4-NOT SPECIFIED|Clerk#000000741|0|slyly slyly even pains. slyly reg|61.5939|-149.43|61.5939|-149.43|
+1986|149|F|34269.96|1994-05-05|2-HIGH|Clerk#000000609|0|across the theodolites. quick|61.5792|-149.495|61.5792|-149.495|
+1987|100|F|6406.29|1994-04-30|2-HIGH|Clerk#000000652|0|gular platelets alongside |61.6141|-149.457|61.6141|-149.457|
+1988|109|O|117132.72|1995-10-06|4-NOT SPECIFIED|Clerk#000000011|0|ly ironic dolphins serve quickly busy accounts. bu|61.5829|-149.448|61.5829|-149.448|
+1989|118|F|39263.28|1994-03-16|4-NOT SPECIFIED|Clerk#000000747|0|ely bold pinto beans ha|61.5938|-149.387|61.5938|-149.387|
+1990|119|F|48781.39|1994-12-16|2-HIGH|Clerk#000000114|0|e bold patterns. always regul|61.5849|-149.38|61.5849|-149.38|
+1991|19|F|139854.41|1992-09-07|4-NOT SPECIFIED|Clerk#000000854|0|ing accounts can haggle at the carefully final Tiresias-- pending, regular|61.5729|-149.389|61.5729|-149.389|
+2016|8|O|24347.36|1996-08-16|3-MEDIUM|Clerk#000000641|0|the carefully ironic foxes. requests nag bold, r|61.5823|-149.462|61.5823|-149.462|
+2017|101|O|70529.27|1998-05-13|3-MEDIUM|Clerk#000000427|0|nusual requests. blit|61.57|-149.331|61.57|-149.331|
+2018|19|P|25007.95|1995-04-05|4-NOT SPECIFIED|Clerk#000000920|0|gular accounts wake fur|61.5821|-149.438|61.5821|-149.438|
+2019|136|F|43789.14|1992-10-23|1-URGENT|Clerk#000000565|0| furiously bold packages. fluffily fi|61.6141|-149.457|61.6141|-149.457|
+2020|73|F|136162.13|1993-06-21|3-MEDIUM|Clerk#000000192|0|es. furiously regular packages above the furiously special theodolites are a|61.6115|-149.331|61.6115|-149.331|
+2021|70|O|27016.74|1995-07-15|1-URGENT|Clerk#000000155|0|ong the furiously regular requests. unusual deposits wake fluffily inside|61.6091|-149.77|61.6091|-149.77|
+2022|62|F|206742.11|1992-03-15|1-URGENT|Clerk#000000268|0| dependencies sleep fluffily even, ironic deposits. express, silen|61.6141|-149.457|61.6141|-149.457|
+2023|118|F|144123.37|1992-05-06|5-LOW|Clerk#000000137|0|ular courts engage according to the|61.5826|-149.427|61.5826|-149.427|
+2048|17|F|33401.77|1993-11-15|1-URGENT|Clerk#000000934|0|s cajole after the blithely final accounts. f|61.5976|-149.366|61.5976|-149.366|
+2049|31|O|153048.74|1995-12-07|2-HIGH|Clerk#000000859|0|ly regular requests thrash blithely about the fluffily even theodolites. r|61.5976|-149.366|61.5976|-149.366|
+2050|28|F|208517.98|1994-06-02|4-NOT SPECIFIED|Clerk#000000821|0|d accounts against the furiously regular packages use bli|61.5531|-149.651|61.5531|-149.651|
+2051|40|O|87988.34|1996-03-18|4-NOT SPECIFIED|Clerk#000000333|0|ctions sleep blithely. blithely regu|61.5531|-149.651|61.5531|-149.651|
+2052|91|F|141822.19|1992-04-13|2-HIGH|Clerk#000000767|0| requests sleep around the even, even courts. ironic theodolites affix furious|61.5883|-149.456|61.5883|-149.456|
+2053|142|F|125125.57|1995-02-07|1-URGENT|Clerk#000000717|0|ar requests: blithely sly accounts boost carefully across t|61.6249|-149.435|61.6249|-149.435|
+2054|41|F|144335.16|1992-06-08|4-NOT SPECIFIED|Clerk#000000103|0|l requests affix carefully about the furiously special|61.6141|-149.457|61.6141|-149.457|
+2055|97|F|57092.26|1993-09-04|1-URGENT|Clerk#000000067|0|. warhorses affix slyly blithely express instructions? fur|61.5709|-149.452|61.5709|-149.452|
+2080|95|F|45767.69|1993-06-18|5-LOW|Clerk#000000190|0|ironic, pending theodolites are carefully about the quickly regular theodolite|61.6651|-149.465|61.6651|-149.465|
+2081|121|O|145654.97|1997-07-05|2-HIGH|Clerk#000000136|0|ong the regular theo|61.5841|-149.441|61.5841|-149.441|
+2082|49|F|46753.63|1995-01-10|2-HIGH|Clerk#000000354|0|cial accounts. ironic, express dolphins nod slyly sometimes final reques|61.1571|-149.883|61.1571|-149.883|
+2083|101|F|31795.52|1993-07-14|3-MEDIUM|Clerk#000000361|0|al patterns. bold, final foxes nag bravely about the furiously express|61.2198|-149.733|61.2198|-149.733|
+2084|80|F|190652.53|1993-03-17|2-HIGH|Clerk#000000048|0|zle furiously final, careful packages. slyly ironic ideas amo|61.1863|-149.976|61.1863|-149.976|
+2085|49|F|45311.07|1993-11-21|3-MEDIUM|Clerk#000000818|0|ress, express ideas haggle|61.2161|-149.876|61.2161|-149.876|
+2086|142|F|188985.18|1994-10-19|1-URGENT|Clerk#000000046|0| permanently regular|61.2031|-149.749|61.2031|-149.749|
+2087|50|O|53581.41|1998-01-31|2-HIGH|Clerk#000000626|0|e always regular packages nod against the furiously spec|61.1644|-149.897|61.1644|-149.897|
+2112|64|O|17986.15|1997-02-05|2-HIGH|Clerk#000000351|0|against the slyly even id|61.1834|-149.866|61.1834|-149.866|
+2113|32|O|65678.21|1997-11-08|2-HIGH|Clerk#000000527|0|slyly regular instruct|61.1731|-149.889|61.1731|-149.889|
+2114|79|F|106446.02|1995-01-16|5-LOW|Clerk#000000751|0|r, unusual accounts haggle across the busy platelets. carefully |61.1089|-149.854|61.1089|-149.854|
+2115|106|O|134814.65|1998-05-23|4-NOT SPECIFIED|Clerk#000000101|0|odolites boost. carefully regular excuses cajole. quickly ironic pinto be|61.1951|-149.916|61.1951|-149.916|
+2116|23|F|60887.90|1994-08-26|1-URGENT|Clerk#000000197|0|efully after the asymptotes. furiously sp|61.2157|-149.821|61.2157|-149.821|
+2117|22|O|145713.03|1997-04-26|2-HIGH|Clerk#000000887|0|ely even dependencies. regular foxes use blithely.|61.1372|-149.954|61.1372|-149.954|
+2118|134|O|38974.67|1996-10-09|1-URGENT|Clerk#000000196|0|ial requests wake carefully special packages. f|61.1955|-149.737|61.1955|-149.737|
+2119|64|O|34632.57|1996-08-20|2-HIGH|Clerk#000000434|0|uickly pending escapades. fluffily ir|61.1444|-149.867|61.1444|-149.867|
+2144|136|F|119917.28|1994-03-29|3-MEDIUM|Clerk#000000546|0|t. carefully quick requests across the deposits wake regu|61.2178|-149.882|61.2178|-149.882|
+2145|134|F|18885.35|1992-10-03|1-URGENT|Clerk#000000886|0|sts would snooze blithely alongside of th|61.1824|-149.849|61.1824|-149.849|
+2146|118|F|179686.07|1992-09-14|4-NOT SPECIFIED|Clerk#000000476|0|ven packages. dependencies wake slyl|61.2161|-149.876|61.2161|-149.876|
+2147|100|F|91513.79|1992-09-06|4-NOT SPECIFIED|Clerk#000000424|0| haggle carefully furiously final foxes. pending escapades thrash. bold theod|61.2022|-149.84|61.2022|-149.84|
+2148|130|F|19612.03|1995-04-19|4-NOT SPECIFIED|Clerk#000000517|0|ross the furiously unusual theodolites. always expre|61.2099|-149.762|61.2099|-149.762|
+2149|101|F|105145.40|1993-03-13|5-LOW|Clerk#000000555|0|nusual accounts nag furiously special reques|61.1951|-149.84|61.1951|-149.84|
+2150|82|F|166961.06|1994-06-03|3-MEDIUM|Clerk#000000154|0|ect slyly against the even, final packages. quickly regular pinto beans wake c|61.1069|-149.859|61.1069|-149.859|
+2151|58|O|124608.69|1996-11-11|3-MEDIUM|Clerk#000000996|0|c requests. ironic platelets cajole across the quickly fluffy deposits.|61.1635|-149.881|61.1635|-149.881|
+2176|104|F|87248.17|1992-11-10|1-URGENT|Clerk#000000195|0|s haggle regularly accor|61.1201|-149.89|61.1201|-149.89|
+2177|136|O|183493.42|1997-01-20|3-MEDIUM|Clerk#000000161|0|ove the blithely unusual packages cajole carefully fluffily special request|61.1902|-149.908|61.1902|-149.908|
+2178|8|O|79594.68|1996-12-12|3-MEDIUM|Clerk#000000656|0|thely according to the instructions. furious|61.2104|-149.857|61.2104|-149.857|
+2179|41|O|77487.09|1996-09-07|2-HIGH|Clerk#000000935|0|ounts alongside of the furiously unusual braids cajol|61.1771|-149.97|61.1771|-149.97|
+2180|76|O|208481.57|1996-09-14|4-NOT SPECIFIED|Clerk#000000650|0|xpress, unusual pains. furiously ironic excu|61.1859|-149.976|61.1859|-149.976|
+2181|76|O|100954.64|1995-09-13|3-MEDIUM|Clerk#000000814|0|y against the ironic, even|61.2171|-149.9|61.2171|-149.9|
+2182|23|F|116003.11|1994-04-05|2-HIGH|Clerk#000000071|0|ccounts. quickly bold deposits across the excuses sl|61.1162|-149.755|61.1162|-149.755|
+2183|113|O|49841.12|1996-06-22|1-URGENT|Clerk#000000287|0| among the express, ironic packages. slyly ironic platelets integrat|61.1381|-149.844|61.1381|-149.844|
+2208|68|P|245388.06|1995-05-01|4-NOT SPECIFIED|Clerk#000000900|0|symptotes wake slyly blithely unusual packages.|61.1775|-149.941|61.1775|-149.941|
+2209|91|F|129086.93|1992-07-10|2-HIGH|Clerk#000000056|0|er above the slyly silent requests. furiously reg|61.1938|-149.878|61.1938|-149.878|
+2210|32|F|31689.46|1992-01-16|2-HIGH|Clerk#000000941|0| believe carefully quickly express pinto beans. deposi|61.1571|-149.883|61.1571|-149.883|
+2211|92|F|140031.23|1994-06-30|2-HIGH|Clerk#000000464|0|ffily bold courts e|61.1541|-149.958|61.1541|-149.958|
+2212|118|F|17231.05|1994-03-23|3-MEDIUM|Clerk#000000954|0|structions above the unusual requests use fur|61.135|-149.88|61.135|-149.88|
+2213|122|F|146136.10|1993-01-15|4-NOT SPECIFIED|Clerk#000000598|0|osits are carefully reg|61.1101|-149.857|61.1101|-149.857|
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/data/orders-with-locations-part2.txt b/hyracks/hyracks-examples/hyracks-integration-tests/data/orders-with-locations-part2.txt
new file mode 100644
index 0000000..afbd373
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/data/orders-with-locations-part2.txt
@@ -0,0 +1,193 @@
+2214|115|O|150345.63|1998-05-05|3-MEDIUM|Clerk#000000253|0|packages. fluffily even accounts haggle blithely. carefully ironic depen|61.1101|-149.857|61.1101|-149.857|
+2215|40|O|108239.46|1996-06-16|4-NOT SPECIFIED|Clerk#000000817|0|le final, final foxes. quickly regular gifts are carefully deposit|61.1101|-149.857|61.1101|-149.857|
+2240|56|F|174090.30|1992-03-06|4-NOT SPECIFIED|Clerk#000000622|0|accounts against the slyly express foxes are after the slyly regular |61.1101|-149.857|61.1101|-149.857|
+2241|103|F|165219.08|1993-05-11|1-URGENT|Clerk#000000081|0|y about the silent excuses. furiously ironic instructions along the sil|61.1101|-149.857|61.1101|-149.857|
+2242|82|O|15082.82|1997-07-20|4-NOT SPECIFIED|Clerk#000000360|0| pending multipliers. carefully express asymptotes use quickl|61.1101|-149.857|61.1101|-149.857|
+2243|49|O|10451.97|1995-06-10|2-HIGH|Clerk#000000813|0|ously regular deposits integrate s|61.1101|-149.857|61.1101|-149.857|
+2244|127|F|21207.08|1993-01-09|1-URGENT|Clerk#000001000|0|ckages. ironic, ironic accounts haggle blithely express excuses. |61.1101|-149.857|61.1101|-149.857|
+2245|58|F|150585.73|1993-04-28|3-MEDIUM|Clerk#000000528|0|ake carefully. braids haggle slyly quickly b|61.1101|-149.857|61.1101|-149.857|
+2246|113|O|85755.84|1996-05-27|4-NOT SPECIFIED|Clerk#000000739|0| final gifts sleep |61.1101|-149.857|61.1101|-149.857|
+2247|95|F|13491.31|1992-08-02|4-NOT SPECIFIED|Clerk#000000947|0|furiously regular packages. final brai|61.1101|-149.857|61.1101|-149.857|
+2272|139|F|127934.71|1993-04-13|2-HIGH|Clerk#000000449|0|s. bold, ironic pinto beans wake. silently specia|61.1101|-149.857|61.1101|-149.857|
+2273|136|O|142291.79|1996-12-14|5-LOW|Clerk#000000155|0|uickly express foxes haggle quickly against|61.1101|-149.857|61.1101|-149.857|
+2274|104|F|58273.89|1993-09-04|4-NOT SPECIFIED|Clerk#000000258|0|nstructions try to hag|61.1101|-149.857|61.1101|-149.857|
+2275|149|F|37398.90|1992-10-22|4-NOT SPECIFIED|Clerk#000000206|0| furiously furious platelets. slyly final packa|61.1101|-149.857|61.1101|-149.857|
+2276|43|O|141159.63|1996-04-29|4-NOT SPECIFIED|Clerk#000000821|0|ecial requests. fox|61.1101|-149.857|61.1101|-149.857|
+2277|89|F|79270.23|1995-01-02|4-NOT SPECIFIED|Clerk#000000385|0|accounts cajole. even i|61.1101|-149.857|61.1101|-149.857|
+2278|142|O|101878.46|1998-04-25|3-MEDIUM|Clerk#000000186|0|r pinto beans integrate after the carefully even deposits. blit|61.1101|-149.857|61.1101|-149.857|
+2279|80|F|142322.33|1993-02-23|3-MEDIUM|Clerk#000000898|0|de of the quickly unusual instructio|61.2141|-149.864|61.2141|-149.864|
+2304|46|F|93769.28|1994-01-07|4-NOT SPECIFIED|Clerk#000000415|0|onic platelets. ironic packages haggle. packages nag doggedly according to|61.2171|-149.9|61.2171|-149.9|
+2305|43|F|122964.66|1993-01-26|2-HIGH|Clerk#000000440|0|ove the furiously even acco|61.2171|-149.9|61.2171|-149.9|
+2306|28|O|244704.23|1995-07-26|2-HIGH|Clerk#000000975|0| wake furiously requests. permanent requests affix. final packages caj|61.2171|-149.9|61.2171|-149.9|
+2307|106|F|59417.76|1993-06-29|5-LOW|Clerk#000000952|0|furiously even asymptotes? carefully regular accounts|61.2171|-149.9|61.2171|-149.9|
+2308|25|F|58546.02|1992-10-25|4-NOT SPECIFIED|Clerk#000000609|0|ts. slyly final depo|61.2171|-149.9|61.2171|-149.9|
+2309|100|O|146933.07|1995-09-04|5-LOW|Clerk#000000803|0|he carefully pending packages. fluffily stealthy foxes engage carefully|61.2171|-149.9|61.2171|-149.9|
+2310|31|O|82928.12|1996-09-20|5-LOW|Clerk#000000917|0|wake carefully. unusual instructions nag ironic, regular excuse|61.2171|-149.9|61.2171|-149.9|
+2311|73|P|153233.93|1995-05-02|2-HIGH|Clerk#000000761|0|ly pending asymptotes-- furiously bold excus|61.2171|-149.9|61.2171|-149.9|
+2336|142|O|22294.51|1996-01-07|4-NOT SPECIFIED|Clerk#000000902|0|c, final excuses sleep furiously among the even theodolites. f|61.2171|-149.9|61.2171|-149.9|
+2337|142|O|45704.96|1997-06-18|4-NOT SPECIFIED|Clerk#000000754|0| quickly. final accounts haggle. carefully final acco|61.2171|-149.9|61.2171|-149.9|
+2338|140|O|28155.92|1997-09-15|2-HIGH|Clerk#000000951|0|riously final dugouts. final, ironic packages wake express, ironic id|61.2171|-149.9|61.2171|-149.9|
+2339|109|F|63470.78|1993-12-15|5-LOW|Clerk#000000847|0| against the regular |61.2171|-149.9|61.2171|-149.9|
+2340|65|O|30778.78|1996-01-12|1-URGENT|Clerk#000000964|0|ter the deposits sleep according to the slyly regular packages. carefully |61.2171|-149.9|61.2171|-149.9|
+2341|82|F|55950.21|1993-05-30|5-LOW|Clerk#000000443|0|sts-- blithely bold dolphins through the deposits nag blithely carefully re|61.2171|-149.9|61.2171|-149.9|
+2342|37|O|104038.78|1996-06-09|1-URGENT|Clerk#000000615|0|oost carefully across the regular accounts. blithely final d|61.2171|-149.9|61.2171|-149.9|
+2343|73|O|85381.00|1995-08-21|3-MEDIUM|Clerk#000000170|0|fluffily over the slyly special deposits. quickl|64.8487|-147.704|64.8487|-147.704|
+2368|13|F|101240.96|1993-08-20|1-URGENT|Clerk#000000830|0|t the bold instructions. carefully unusual |64.8486|-147.705|64.8486|-147.705|
+2369|110|O|73517.91|1996-12-24|2-HIGH|Clerk#000000752|0|iously even requests are dogged, express |64.8087|-147.71|64.8087|-147.71|
+2370|142|F|73924.21|1994-01-17|1-URGENT|Clerk#000000231|0|lyly final packages. quickly final deposits haggl|64.8363|-147.758|64.8363|-147.758|
+2371|19|O|193857.67|1998-01-07|1-URGENT|Clerk#000000028|0|ckages haggle at th|64.8476|-147.704|64.8476|-147.704|
+2372|31|O|104927.66|1997-11-21|5-LOW|Clerk#000000342|0|s: deposits haggle along the final ideas. careful|64.8302|-147.744|64.8302|-147.744|
+2373|28|F|55211.04|1994-03-12|4-NOT SPECIFIED|Clerk#000000306|0| even, special courts grow quickly. pending,|64.8476|-147.812|64.8476|-147.812|
+2374|4|F|115219.88|1993-10-29|4-NOT SPECIFIED|Clerk#000000081|0| blithely regular packages. blithely unusua|64.8144|-147.756|64.8144|-147.756|
+2375|5|O|106612.48|1996-11-20|3-MEDIUM|Clerk#000000197|0|unusual, pending theodolites cajole carefully |64.8183|-147.778|64.8183|-147.778|
+2400|37|O|92798.66|1998-07-25|5-LOW|Clerk#000000782|0|nusual courts nag against the carefully unusual pinto b|64.8494|-147.818|64.8494|-147.818|
+2401|148|O|88448.24|1997-07-29|4-NOT SPECIFIED|Clerk#000000531|0|ully unusual instructions boost carefully silently regular requests. |64.849|-147.822|64.849|-147.822|
+2402|67|O|70403.62|1996-09-06|4-NOT SPECIFIED|Clerk#000000162|0|slyly final sheaves sleep slyly. q|64.8367|-147.716|64.8367|-147.716|
+2403|55|O|111020.79|1998-04-11|3-MEDIUM|Clerk#000000820|0|furiously regular deposits use. furiously unusual accounts wake along the |64.8127|-147.772|64.8127|-147.772|
+2404|77|O|109077.69|1997-03-13|4-NOT SPECIFIED|Clerk#000000409|0|deposits breach furiously. ironic foxes haggle carefully bold packag|64.8143|-147.751|64.8143|-147.751|
+2405|73|O|115929.14|1996-12-23|3-MEDIUM|Clerk#000000535|0|ular, regular asympto|64.842|-147.721|64.842|-147.721|
+2406|7|O|182516.77|1996-10-28|5-LOW|Clerk#000000561|0|blithely regular accounts u|64.8403|-147.714|64.8403|-147.714|
+2407|55|O|112843.52|1998-06-19|2-HIGH|Clerk#000000068|0|uests affix slyly among the slyly regular depos|64.8371|-147.881|64.8371|-147.881|
+2432|103|O|62661.93|1996-07-13|1-URGENT|Clerk#000000115|0|re. slyly even deposits wake bra|64.8151|-147.707|64.8151|-147.707|
+2433|31|F|147071.86|1994-08-22|4-NOT SPECIFIED|Clerk#000000324|0|ess patterns are slyly. packages haggle carefu|64.8151|-147.707|64.8151|-147.707|
+2434|25|O|123956.25|1997-04-27|3-MEDIUM|Clerk#000000190|0|s. quickly ironic dolphins impress final deposits. blithel|64.8541|-147.81|64.8541|-147.81|
+2435|73|F|122490.66|1993-02-21|5-LOW|Clerk#000000112|0|es are carefully along the carefully final instructions. pe|64.8878|-147.496|64.8878|-147.496|
+2436|125|O|73990.08|1995-09-11|4-NOT SPECIFIED|Clerk#000000549|0|arefully. blithely bold deposits affix special accounts. final foxes nag. spe|64.8299|-147.728|64.8299|-147.728|
+2437|85|F|143411.69|1993-04-21|4-NOT SPECIFIED|Clerk#000000578|0|. theodolites wake slyly-- ironic, pending platelets above the carefully exp|64.8132|-147.762|64.8132|-147.762|
+2438|13|F|214494.39|1993-07-15|2-HIGH|Clerk#000000744|0|the final, regular warhorses. regularly |64.8372|-147.713|64.8372|-147.713|
+2439|55|O|41811.12|1997-03-15|2-HIGH|Clerk#000000819|0|lithely after the car|64.7927|-148.036|64.7927|-148.036|
+2464|145|O|30495.65|1997-11-23|5-LOW|Clerk#000000633|0|le about the instructions. courts wake carefully even|64.8717|-147.819|64.8717|-147.819|
+2465|34|O|180737.75|1995-06-24|1-URGENT|Clerk#000000078|0|al pinto beans. final, bold packages wake quickly|64.8527|-147.686|64.8527|-147.686|
+2466|19|F|161625.50|1994-03-06|1-URGENT|Clerk#000000424|0|c pinto beans. express deposits wake quickly. even, final courts nag. package|64.8371|-147.811|64.8371|-147.811|
+2467|35|O|7231.91|1995-07-16|4-NOT SPECIFIED|Clerk#000000914|0|pades sleep furiously. sometimes regular packages again|64.846|-147.705|64.846|-147.705|
+2468|112|O|160627.01|1997-06-09|4-NOT SPECIFIED|Clerk#000000260|0|ickly regular packages. slyly ruthless requests snooze quickly blithe|64.9064|-147.726|64.9064|-147.726|
+2469|124|O|192074.23|1996-11-26|5-LOW|Clerk#000000730|0| sleep closely regular instructions. furiously ironic instructi|64.9347|-147.56|64.9347|-147.56|
+2470|58|O|104966.33|1997-04-19|3-MEDIUM|Clerk#000000452|0|to the furiously final packages? pa|64.8861|-147.677|64.8861|-147.677|
+2471|89|O|34936.31|1998-03-12|4-NOT SPECIFIED|Clerk#000000860|0|carefully blithely regular pac|64.8302|-147.744|64.8302|-147.744|
+2496|136|F|140390.60|1994-01-09|2-HIGH|Clerk#000000142|0|slyly. pending instructions sleep. quic|60.6673|-151.311|60.6673|-151.311|
+2497|47|F|171326.48|1992-08-27|1-URGENT|Clerk#000000977|0|ily ironic pinto beans. furiously final platelets alongside of t|60.6997|-151.38|60.6997|-151.38|
+2498|97|F|45514.27|1993-11-08|5-LOW|Clerk#000000373|0|g the slyly special pinto beans. |60.5658|-151.244|60.5658|-151.244|
+2499|121|O|147243.86|1995-09-24|1-URGENT|Clerk#000000277|0|r the quickly bold foxes. bold instructi|60.6331|-151.163|60.6331|-151.163|
+2500|133|F|131122.82|1992-08-15|2-HIGH|Clerk#000000447|0|integrate slyly pending deposits. furiously ironic accounts across the s|60.6331|-151.163|60.6331|-151.163|
+2501|67|O|79380.51|1997-05-25|5-LOW|Clerk#000000144|0|ickly special theodolite|60.6331|-151.163|60.6331|-151.163|
+2502|70|F|33470.40|1993-05-28|4-NOT SPECIFIED|Clerk#000000914|0|lyly: carefully pending ideas affix again|60.6201|-151.332|60.6201|-151.332|
+2503|7|F|183671.08|1993-06-20|3-MEDIUM|Clerk#000000294|0|ly even packages was. ironic, regular deposits unwind furiously across the p|60.5004|-151.276|60.5004|-151.276|
+2528|55|F|92069.62|1994-11-20|1-URGENT|Clerk#000000789|0|ular dependencies? regular frays kindle according to the blith|60.6331|-151.163|60.6331|-151.163|
+2529|136|O|4104.30|1996-08-20|2-HIGH|Clerk#000000511|0|posits across the silent instructions wake blithely across |60.6331|-151.163|60.6331|-151.163|
+2530|128|F|58853.11|1994-03-21|3-MEDIUM|Clerk#000000291|0|ular instructions about the quic|60.6901|-151.321|60.6901|-151.321|
+2531|44|O|143212.85|1996-05-06|4-NOT SPECIFIED|Clerk#000000095|0|even accounts. furiously ironic excuses sleep fluffily. carefully silen|60.6676|-151.29|60.6676|-151.29|
+2532|94|O|116093.49|1995-10-11|2-HIGH|Clerk#000000498|0|the blithely pending accounts. regular, regular excuses boost aro|60.6331|-151.163|60.6331|-151.163|
+2533|50|O|168495.03|1997-03-24|1-URGENT|Clerk#000000594|0|ecial instructions. spec|60.5632|-151.266|60.5632|-151.266|
+2534|76|O|202784.54|1996-07-17|3-MEDIUM|Clerk#000000332|0|packages cajole ironic requests. furiously regular|60.6331|-151.163|60.6331|-151.163|
+2535|121|F|67018.30|1993-05-25|5-LOW|Clerk#000000296|0|phins cajole beneath the fluffily express asymptotes. c|60.6331|-151.163|60.6331|-151.163|
+2560|131|F|153426.79|1992-09-05|1-URGENT|Clerk#000000538|0|atelets; quickly sly requests|60.6509|-151.342|60.6509|-151.342|
+2561|58|O|137473.58|1997-11-14|1-URGENT|Clerk#000000861|0|ual requests. unusual deposits cajole furiously pending, regular platelets. |60.5601|-151.107|60.5601|-151.107|
+2562|10|F|136360.37|1992-08-01|1-URGENT|Clerk#000000467|0|elets. pending dolphins promise slyly. bo|60.5123|-151.275|60.5123|-151.275|
+2563|62|F|168952.10|1993-11-19|4-NOT SPECIFIED|Clerk#000000150|0|sly even packages after the furio|60.6076|-151.325|60.6076|-151.325|
+2564|77|F|3967.47|1994-09-09|2-HIGH|Clerk#000000718|0|usly regular pinto beans. orbits wake carefully. slyly e|60.6331|-151.163|60.6331|-151.163|
+2565|56|O|204438.57|1998-02-28|3-MEDIUM|Clerk#000000032|0|x-ray blithely along|60.5175|-151.235|60.5175|-151.235|
+2566|86|F|89992.48|1992-10-10|3-MEDIUM|Clerk#000000414|0|ructions boost bold ideas. idly ironic accounts use according to th|60.5535|-151.108|60.5535|-151.108|
+2567|70|O|263411.29|1998-02-27|2-HIGH|Clerk#000000031|0|detect. furiously ironic requests|60.5614|-151.275|60.5614|-151.275|
+2592|101|F|8225.96|1993-03-05|4-NOT SPECIFIED|Clerk#000000524|0|ts nag fluffily. quickly stealthy theodolite|60.5647|-151.195|60.5647|-151.195|
+2593|92|F|134726.09|1993-09-04|2-HIGH|Clerk#000000468|0|r the carefully final|60.6331|-151.163|60.6331|-151.163|
+2594|79|F|94866.39|1992-12-17|1-URGENT|Clerk#000000550|0|ests. theodolites above the blithely even accounts detect furio|60.6331|-151.163|60.6331|-151.163|
+2595|74|O|173130.20|1995-12-14|4-NOT SPECIFIED|Clerk#000000222|0|arefully ironic requests nag carefully ideas. |60.6331|-151.163|60.6331|-151.163|
+2596|43|O|74940.13|1996-08-17|1-URGENT|Clerk#000000242|0|requests. ironic, bold theodolites wak|60.6331|-151.163|60.6331|-151.163|
+2597|104|F|21964.66|1993-02-04|2-HIGH|Clerk#000000757|0|iously ruthless exc|60.6331|-151.163|60.6331|-151.163|
+2598|112|O|84871.50|1996-03-05|3-MEDIUM|Clerk#000000391|0| ironic notornis according to the blithely final requests should |60.6678|-151.31|60.6678|-151.31|
+2599|149|O|62807.13|1996-11-07|2-HIGH|Clerk#000000722|0|ts. slyly regular theodolites wake sil|60.5003|-151.276|60.5003|-151.276|
+2624|52|O|27148.63|1996-11-28|5-LOW|Clerk#000000930|0|ic, regular packages|60.6331|-151.163|60.6331|-151.163|
+2625|40|F|39382.74|1992-10-14|4-NOT SPECIFIED|Clerk#000000386|0| final deposits. blithely ironic ideas |61.5855|-149.326|61.5855|-149.326|
+2626|139|O|84314.51|1995-09-08|4-NOT SPECIFIED|Clerk#000000289|0|gside of the carefully special packages are furiously after the slyly express |61.5979|-149.437|61.5979|-149.437|
+2627|149|F|26798.65|1992-03-24|3-MEDIUM|Clerk#000000181|0|s. silent, ruthless requests|61.6141|-149.457|61.6141|-149.457|
+2628|56|F|165655.99|1993-10-22|5-LOW|Clerk#000000836|0|ajole across the blithely careful accounts. blithely silent deposits sl|61.5799|-149.461|61.5799|-149.461|
+2629|139|O|96458.03|1998-04-06|5-LOW|Clerk#000000680|0|uches dazzle carefully even, express excuses. ac|61.5845|-149.337|61.5845|-149.337|
+2630|85|F|127132.51|1992-10-24|5-LOW|Clerk#000000712|0|inal theodolites. ironic instructions s|61.5351|-149.558|61.5351|-149.558|
+2631|37|F|63103.32|1993-09-24|5-LOW|Clerk#000000833|0| quickly unusual deposits doubt around |61.5811|-149.45|61.5811|-149.45|
+2656|77|F|105492.37|1993-05-04|1-URGENT|Clerk#000000307|0|elets. slyly final accou|61.5793|-149.442|61.5793|-149.442|
+2657|25|O|148176.06|1995-10-17|2-HIGH|Clerk#000000160|0| foxes-- slyly final dependencies around the slyly final theodo|61.5661|-149.313|61.5661|-149.313|
+2658|14|O|163834.46|1995-09-23|3-MEDIUM|Clerk#000000400|0|bout the slyly regular accounts. ironic, |61.6141|-149.457|61.6141|-149.457|
+2659|83|F|79785.52|1993-12-18|4-NOT SPECIFIED|Clerk#000000758|0|cross the pending requests maintain |61.5786|-149.332|61.5786|-149.332|
+2660|127|O|16922.51|1995-08-05|5-LOW|Clerk#000000480|0|ly finally regular deposits. ironic theodolites cajole|61.5811|-149.45|61.5811|-149.45|
+2661|74|O|106036.84|1997-01-04|3-MEDIUM|Clerk#000000217|0|al, regular pinto beans. silently final deposits should have t|61.5825|-149.429|61.5825|-149.429|
+2662|37|O|87689.88|1996-08-21|3-MEDIUM|Clerk#000000589|0|bold pinto beans above the slyly final accounts affix furiously deposits. pac|61.6141|-149.457|61.6141|-149.457|
+2663|95|O|35131.80|1995-09-06|1-URGENT|Clerk#000000950|0|ar requests. furiously final dolphins along the fluffily spe|61.5531|-149.651|61.5531|-149.651|
+2688|98|F|181077.36|1992-01-24|2-HIGH|Clerk#000000720|0|have to nag according to the pending theodolites. sly|61.5531|-149.651|61.5531|-149.651|
+2689|103|F|41552.78|1992-04-09|4-NOT SPECIFIED|Clerk#000000698|0|press pains wake. furiously express theodolites alongsid|61.5698|-149.62|61.5698|-149.62|
+2690|94|O|224674.27|1996-03-31|3-MEDIUM|Clerk#000000760|0|ravely even theodolites |61.6141|-149.457|61.6141|-149.457|
+2691|7|F|30137.17|1992-04-30|5-LOW|Clerk#000000439|0|es at the regular deposits sleep slyly by the fluffy requests. eve|61.5474|-149.458|61.5474|-149.458|
+2692|62|O|24265.24|1997-12-02|3-MEDIUM|Clerk#000000878|0|es. regular asymptotes cajole above t|61.5825|-149.429|61.5825|-149.429|
+2693|19|O|66158.13|1996-09-04|1-URGENT|Clerk#000000370|0|ndle never. blithely regular packages nag carefully enticing platelets. ca|61.5955|-149.423|61.5955|-149.423|
+2694|121|O|102807.59|1996-03-14|5-LOW|Clerk#000000722|0| requests. bold deposits above the theodol|61.5801|-149.461|61.5801|-149.461|
+2695|58|O|138584.20|1996-08-20|1-URGENT|Clerk#000000697|0|ven deposits around the quickly regular packa|61.5785|-149.415|61.5785|-149.415|
+2720|31|F|161307.05|1993-06-08|1-URGENT|Clerk#000000948|0|quickly. special asymptotes are fluffily ironi|61.6402|-149.34|61.6402|-149.34|
+2721|79|O|59180.25|1996-01-27|2-HIGH|Clerk#000000401|0| ideas eat even, unusual ideas. theodolites are carefully|61.583|-149.457|61.583|-149.457|
+2722|35|F|50328.84|1994-04-09|5-LOW|Clerk#000000638|0|rding to the carefully quick deposits. bli|61.5907|-149.295|61.5907|-149.295|
+2723|61|O|104759.25|1995-10-06|5-LOW|Clerk#000000836|0|nts must have to cajo|61.6141|-149.457|61.6141|-149.457|
+2724|137|F|116069.66|1994-09-14|2-HIGH|Clerk#000000217|0| sleep blithely. blithely idle |61.5933|-149.397|61.5933|-149.397|
+2725|89|F|75144.68|1994-05-21|4-NOT SPECIFIED|Clerk#000000835|0|ular deposits. spec|61.6091|-149.77|61.6091|-149.77|
+2726|7|F|47753.00|1992-11-27|5-LOW|Clerk#000000470|0| blithely even dinos sleep care|61.577|-149.411|61.577|-149.411|
+2727|74|O|3089.42|1998-04-19|4-NOT SPECIFIED|Clerk#000000879|0|sual theodolites cajole enticingly above the furiously fin|61.6078|-149.322|61.6078|-149.322|
+2752|59|F|187932.30|1993-11-19|2-HIGH|Clerk#000000648|0| carefully regular foxes are quickly quickl|61.6131|-149.397|61.6131|-149.397|
+2753|16|F|159720.39|1993-11-30|2-HIGH|Clerk#000000380|0|ending instructions. unusual deposits|61.6648|-149.372|61.6648|-149.372|
+2754|145|F|25985.52|1994-04-03|2-HIGH|Clerk#000000960|0|cies detect slyly. |61.5531|-149.651|61.5531|-149.651|
+2755|118|F|101202.18|1992-02-07|4-NOT SPECIFIED|Clerk#000000177|0|ously according to the sly foxes. blithely regular pinto bean|61.5811|-149.45|61.5811|-149.45|
+2756|118|F|142323.38|1994-04-18|1-URGENT|Clerk#000000537|0|arefully special warho|61.583|-149.457|61.583|-149.457|
+2757|76|O|89792.48|1995-07-20|2-HIGH|Clerk#000000216|0| regular requests subl|61.1955|-149.9|61.1955|-149.9|
+2758|43|O|36671.88|1998-07-12|5-LOW|Clerk#000000863|0|s cajole according to the carefully special |61.1844|-149.897|61.1844|-149.897|
+2759|116|F|89731.10|1993-11-25|4-NOT SPECIFIED|Clerk#000000071|0|ts. regular, pending pinto beans sleep ab|61.1901|-149.892|61.1901|-149.892|
+2784|95|O|106635.21|1998-01-07|1-URGENT|Clerk#000000540|0|g deposits alongside of the silent requests s|61.1444|-149.867|61.1444|-149.867|
+2785|148|O|132854.79|1995-07-21|2-HIGH|Clerk#000000098|0|iously pending packages sleep according to the blithely unusual foxe|61.1955|-149.9|61.1955|-149.9|
+2786|79|F|178254.66|1992-03-22|2-HIGH|Clerk#000000976|0|al platelets cajole blithely ironic requests. ironic re|61.1893|-149.887|61.1893|-149.887|
+2787|103|O|3726.14|1995-09-30|1-URGENT|Clerk#000000906|0|he ironic, regular |61.2174|-149.888|61.2174|-149.888|
+2788|124|F|17172.66|1994-09-22|1-URGENT|Clerk#000000641|0|nts wake across the fluffily bold accoun|61.2227|-149.842|61.2227|-149.842|
+2789|37|O|219123.27|1998-03-14|2-HIGH|Clerk#000000972|0|gular patterns boost. carefully even re|61.1263|-149.872|61.1263|-149.872|
+2790|25|F|177458.97|1994-08-19|2-HIGH|Clerk#000000679|0| the carefully express deposits sleep slyly |61.1138|-149.866|61.1138|-149.866|
+2791|121|F|156697.55|1994-10-10|2-HIGH|Clerk#000000662|0|as. slyly ironic accounts play furiously bl|61.2157|-149.821|61.2157|-149.821|
+2816|58|F|42225.53|1994-09-20|2-HIGH|Clerk#000000289|0|kages at the final deposits cajole furious foxes. quickly |61.2174|-149.888|61.2174|-149.888|
+2817|40|F|71453.85|1994-04-19|3-MEDIUM|Clerk#000000982|0|ic foxes haggle upon the daringly even pinto beans. slyly|61.1855|-149.868|61.1855|-149.868|
+2818|49|F|120086.84|1994-12-12|3-MEDIUM|Clerk#000000413|0|eep furiously special ideas. express |61.1951|-149.873|61.1951|-149.873|
+2819|103|F|66927.16|1994-05-05|1-URGENT|Clerk#000000769|0|ngside of the blithely ironic dolphins. furio|61.1444|-149.867|61.1444|-149.867|
+2820|19|F|143813.39|1994-05-20|3-MEDIUM|Clerk#000000807|0|equests are furiously. carefu|61.1883|-149.735|61.1883|-149.735|
+2821|118|F|36592.48|1993-08-09|3-MEDIUM|Clerk#000000323|0|ng requests. even instructions are quickly express, silent instructi|61.2161|-149.876|61.2161|-149.876|
+2822|79|F|40142.15|1993-07-26|2-HIGH|Clerk#000000510|0|furiously against the accounts. unusual accounts aft|61.2161|-149.876|61.2161|-149.876|
+2823|79|O|171894.45|1995-09-09|2-HIGH|Clerk#000000567|0|encies. carefully fluffy accounts m|61.1893|-149.888|61.1893|-149.888|
+2848|70|F|116258.53|1992-03-10|1-URGENT|Clerk#000000256|0|ly fluffy foxes sleep furiously across the slyly regu|61.2174|-149.888|61.2174|-149.888|
+2849|46|O|180054.29|1996-04-30|2-HIGH|Clerk#000000659|0|al packages are after the quickly bold requests. carefully special |61.1914|-149.886|61.1914|-149.886|
+2850|100|O|122969.79|1996-10-02|2-HIGH|Clerk#000000392|0|, regular deposits. furiously pending packages hinder carefully carefully u|61.1541|-149.958|61.1541|-149.958|
+2851|145|O|7859.36|1997-09-07|5-LOW|Clerk#000000566|0|Tiresias wake quickly quickly even|61.1259|-149.717|61.1259|-149.717|
+2852|91|F|99050.81|1993-01-16|1-URGENT|Clerk#000000740|0|ruthless deposits against the final instructions use quickly al|61.2193|-149.902|61.2193|-149.902|
+2853|94|F|103641.15|1994-05-05|2-HIGH|Clerk#000000878|0|the carefully even packages.|61.1879|-149.886|61.1879|-149.886|
+2854|139|F|153568.02|1994-06-27|1-URGENT|Clerk#000000010|0| furiously ironic tithes use furiously |61.1372|-149.912|61.1372|-149.912|
+2855|49|F|48419.58|1993-04-04|4-NOT SPECIFIED|Clerk#000000973|0| silent, regular packages sleep |61.1101|-149.857|61.1101|-149.857|
+2880|8|F|145761.99|1992-03-15|2-HIGH|Clerk#000000756|0|ves maintain doggedly spec|61.1791|-149.94|61.1791|-149.94|
+2881|100|F|45695.84|1992-05-10|5-LOW|Clerk#000000864|0|uriously. slyly express requests according to the silent dol|61.2031|-149.749|61.2031|-149.749|
+2882|121|O|172872.37|1995-08-22|2-HIGH|Clerk#000000891|0|pending deposits. carefully eve|61.1914|-149.877|61.1914|-149.877|
+2883|121|F|170360.27|1995-01-23|5-LOW|Clerk#000000180|0|uses. carefully ironic accounts lose fluffil|61.1944|-149.883|61.1944|-149.883|
+2884|92|O|71683.84|1997-10-12|3-MEDIUM|Clerk#000000780|0|efully express instructions sleep against|61.1923|-149.886|61.1923|-149.886|
+2885|7|F|146896.72|1992-09-19|4-NOT SPECIFIED|Clerk#000000280|0|ly sometimes special excuses. final requests are |61.2123|-149.854|61.2123|-149.854|
+2886|109|F|94527.23|1994-11-13|4-NOT SPECIFIED|Clerk#000000619|0|uctions. ironic packages sle|61.2161|-149.876|61.2161|-149.876|
+2887|109|O|28571.39|1997-05-26|5-LOW|Clerk#000000566|0|slyly even pinto beans. slyly bold epitaphs cajole blithely above t|61.2171|-149.9|61.2171|-149.9|
+2912|94|F|27727.52|1992-03-12|5-LOW|Clerk#000000186|0|jole blithely above the quickly regular packages. carefully regular pinto bean|61.1125|-149.861|61.1125|-149.861|
+2913|43|O|130702.19|1997-07-12|3-MEDIUM|Clerk#000000118|0|mptotes doubt furiously slyly regu|61.1419|-149.896|61.1419|-149.896|
+2914|109|F|60867.14|1993-03-03|3-MEDIUM|Clerk#000000543|0|he slyly regular theodolites are furiously sile|61.145|-149.878|61.145|-149.878|
+2915|94|F|96015.13|1994-03-31|5-LOW|Clerk#000000410|0|ld packages. bold deposits boost blithely. ironic, unusual theodoli|61.1044|-149.865|61.1044|-149.865|
+2916|8|O|20182.22|1995-12-27|2-HIGH|Clerk#000000681|0|ithely blithe deposits sleep beyond the|61.1444|-149.876|61.1444|-149.876|
+2917|91|O|100714.13|1997-12-09|4-NOT SPECIFIED|Clerk#000000061|0| special dugouts among the special deposi|61.1|-149.85|61.1|-149.85|
+2918|118|O|21760.09|1996-09-08|3-MEDIUM|Clerk#000000439|0|ular deposits across th|61.1105|-149.861|61.1105|-149.861|
+2919|53|F|137223.14|1993-12-10|2-HIGH|Clerk#000000209|0|es. pearls wake quietly slyly ironic instructions--|61.1286|-149.957|61.1286|-149.957|
+2944|14|O|146581.14|1997-09-24|4-NOT SPECIFIED|Clerk#000000740|0|deas. permanently special foxes haggle carefully ab|61.1201|-149.89|61.1201|-149.89|
+2945|29|O|223507.72|1996-01-03|2-HIGH|Clerk#000000499|0|ons are carefully toward the permanent, bold pinto beans. regu|61.112|-149.871|61.112|-149.871|
+2946|125|O|102226.59|1996-02-05|5-LOW|Clerk#000000329|0|g instructions about the regular accounts sleep carefully along the pen|61.1427|-149.864|61.1427|-149.864|
+2947|70|P|43360.95|1995-04-26|1-URGENT|Clerk#000000464|0|ronic accounts. accounts run furiously d|61.1212|-149.947|61.1212|-149.947|
+2948|44|F|100758.71|1994-08-23|5-LOW|Clerk#000000701|0| deposits according to the blithely pending |61.1228|-149.939|61.1228|-149.939|
+2949|137|F|94231.71|1994-04-12|2-HIGH|Clerk#000000184|0|y ironic accounts use. quickly blithe accou|61.1093|-149.871|61.1093|-149.871|
+2950|136|O|183620.33|1997-07-06|1-URGENT|Clerk#000000833|0| dolphins around the furiously |61.145|-149.878|61.145|-149.878|
+2951|74|O|125509.17|1996-02-06|2-HIGH|Clerk#000000680|0|gular deposits above the finally regular ideas integrate idly stealthil|61.1191|-149.871|61.1191|-149.871|
+2976|29|F|145768.47|1993-12-10|4-NOT SPECIFIED|Clerk#000000159|0|. furiously ironic asymptotes haggle ruthlessly silently regular r|61.1003|-149.856|61.1003|-149.856|
+2977|73|O|25170.88|1996-08-27|3-MEDIUM|Clerk#000000252|0|quickly special platelets are furio|61.1113|-149.872|61.1113|-149.872|
+2978|44|P|139542.14|1995-05-03|1-URGENT|Clerk#000000135|0|d. even platelets are. ironic dependencies cajole slow, e|61.1084|-149.861|61.1084|-149.861|
+2979|133|O|116789.98|1996-03-23|3-MEDIUM|Clerk#000000820|0|even, ironic foxes sleep along|61.144|-149.878|61.144|-149.878|
+2980|4|O|187514.11|1996-09-14|3-MEDIUM|Clerk#000000661|0|y quick pinto beans wake. slyly re|61.1426|-149.877|61.1426|-149.877|
+2981|49|O|37776.79|1998-07-29|5-LOW|Clerk#000000299|0|hely among the express foxes. blithely stealthy requests cajole boldly. regu|61.1173|-149.861|61.1173|-149.861|
+2982|85|F|55582.94|1995-03-19|2-HIGH|Clerk#000000402|0|lyly. express theodolites affix slyly after the slyly speci|61.1347|-149.914|61.1347|-149.914|
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/data/orders-with-locations.txt b/hyracks/hyracks-examples/hyracks-integration-tests/data/orders-with-locations.txt
deleted file mode 100644
index f936a6e..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/data/orders-with-locations.txt
+++ /dev/null
@@ -1,750 +0,0 @@
-1|37|O|131251.81|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among |42.3631|-71.065|42.3631|-71.065|
-2|79|O|40183.29|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot|42.1091|-70.696|42.1091|-70.696|
-3|124|F|160882.76|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos|40.8151|-73.0452|40.8151|-73.0452|
-4|137|O|31084.79|1995-10-11|5-LOW|Clerk#000000124|0|sits. slyly regular warthogs cajole. regular, regular theodolites acro|40.8151|-73.0452|40.8151|-73.0452|
-5|46|F|86615.25|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages use slyly|42.2481|-71.174|42.2481|-71.174|
-6|56|F|36468.55|1992-02-21|4-NOT SPECIFIED|Clerk#000000058|0|ggle. special, final requests are against the furiously specia|61.1201|-149.89|61.1201|-149.89|
-7|40|O|171488.73|1996-01-10|2-HIGH|Clerk#000000470|0|ly special requests |61.1501|-149.926|61.1501|-149.926|
-32|131|O|116923.00|1995-07-16|2-HIGH|Clerk#000000616|0|ise blithely bold, regular requests. quickly unusual dep|61.181|-149.814|61.181|-149.814|
-33|67|F|99798.76|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request|61.1517|-149.86|61.1517|-149.86|
-34|62|O|41670.02|1998-07-21|3-MEDIUM|Clerk#000000223|0|ly final packages. fluffily final deposits wake blithely ideas. spe|61.1806|-149.814|61.1806|-149.814|
-35|128|O|148789.52|1995-10-23|4-NOT SPECIFIED|Clerk#000000259|0|zzle. carefully enticing deposits nag furio|61.1806|-149.775|61.1806|-149.775|
-36|116|O|38988.98|1995-11-03|1-URGENT|Clerk#000000358|0| quick packages are blithely. slyly silent accounts wake qu|61.1806|-149.775|61.1806|-149.775|
-37|88|F|113701.89|1992-06-03|3-MEDIUM|Clerk#000000456|0|kly regular pinto beans. carefully unusual waters cajole never|61.1806|-149.775|61.1806|-149.775|
-38|125|O|46366.56|1996-08-21|4-NOT SPECIFIED|Clerk#000000604|0|haggle blithely. furiously express ideas haggle blithely furiously regular re|61.2113|-149.824|61.2113|-149.824|
-39|82|O|219707.84|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir|61.1967|-149.877|61.1967|-149.877|
-64|34|F|20065.73|1994-07-16|3-MEDIUM|Clerk#000000661|0|wake fluffily. sometimes ironic pinto beans about the dolphin|61.2164|-149.892|61.2164|-149.892|
-65|17|P|65883.92|1995-03-18|1-URGENT|Clerk#000000632|0|ular requests are blithely pending orbits-- even requests against the deposit|61.1571|-149.883|61.1571|-149.883|
-66|130|F|79258.24|1994-01-20|5-LOW|Clerk#000000743|0|y pending requests integrate|61.2048|-149.834|61.2048|-149.834|
-67|58|O|116227.05|1996-12-19|4-NOT SPECIFIED|Clerk#000000547|0|symptotes haggle slyly around the furiously iron|61.0956|-149.843|61.0956|-149.843|
-68|29|O|215135.72|1998-04-18|3-MEDIUM|Clerk#000000440|0| pinto beans sleep carefully. blithely ironic deposits haggle furiously acro|61.1491|-149.809|61.1491|-149.809|
-69|85|F|162176.23|1994-06-04|4-NOT SPECIFIED|Clerk#000000330|0| depths atop the slyly thin deposits detect among the furiously silent accou|61.1981|-149.871|61.1981|-149.871|
-70|65|F|84651.80|1993-12-18|5-LOW|Clerk#000000322|0| carefully ironic request|61.1982|-149.876|61.1982|-149.876|
-71|4|O|178821.73|1998-01-24|4-NOT SPECIFIED|Clerk#000000271|0| express deposits along the blithely regul|61.1924|-149.909|61.1924|-149.909|
-96|109|F|55090.67|1994-04-17|2-HIGH|Clerk#000000395|0|oost furiously. pinto|61.2204|-149.728|61.2204|-149.728|
-97|22|F|68908.31|1993-01-29|3-MEDIUM|Clerk#000000547|0|hang blithely along the regular accounts. furiously even ideas after the|61.196|-149.864|61.196|-149.864|
-98|106|F|51004.44|1994-09-25|1-URGENT|Clerk#000000448|0|c asymptotes. quickly regular packages should have to nag re|61.1987|-149.889|61.1987|-149.889|
-99|89|F|92326.79|1994-03-13|4-NOT SPECIFIED|Clerk#000000973|0|e carefully ironic packages. pending|61.1984|-149.897|61.1984|-149.897|
-100|148|O|141311.01|1998-02-28|4-NOT SPECIFIED|Clerk#000000577|0|heodolites detect slyly alongside of the ent|61.1897|-149.898|61.1897|-149.898|
-101|28|O|95591.40|1996-03-17|3-MEDIUM|Clerk#000000419|0|ding accounts above the slyly final asymptote|61.1228|-149.81|61.1228|-149.81|
-102|1|O|113954.89|1997-05-09|2-HIGH|Clerk#000000596|0| slyly according to the asymptotes. carefully final packages integrate furious|61.1649|-149.881|61.1649|-149.881|
-103|31|O|95563.95|1996-06-20|4-NOT SPECIFIED|Clerk#000000090|0|ges. carefully unusual instructions haggle quickly regular f|61.1934|-149.887|61.1934|-149.887|
-128|74|F|36333.34|1992-06-15|1-URGENT|Clerk#000000385|0|ns integrate fluffily. ironic asymptotes after the regular excuses nag around |61.2164|-149.892|61.2164|-149.892|
-129|73|F|188124.55|1992-11-19|5-LOW|Clerk#000000859|0|ing tithes. carefully pending deposits boost about the silently express |61.1932|-149.886|61.1932|-149.886|
-130|37|F|115717.37|1992-05-08|2-HIGH|Clerk#000000036|0|le slyly unusual, regular packages? express deposits det|61.2072|-149.888|61.2072|-149.888|
-131|94|F|96596.81|1994-06-08|3-MEDIUM|Clerk#000000625|0|after the fluffily special foxes integrate s|61.2125|-149.904|61.2125|-149.904|
-132|28|F|118802.62|1993-06-11|3-MEDIUM|Clerk#000000488|0|sits are daringly accounts. carefully regular foxes sleep slyly about the|61.2142|-149.806|61.2142|-149.806|
-133|44|O|80437.72|1997-11-29|1-URGENT|Clerk#000000738|0|usly final asymptotes |61.1866|-149.923|61.1866|-149.923|
-134|7|F|154260.84|1992-05-01|4-NOT SPECIFIED|Clerk#000000711|0|lar theodolites boos|61.1089|-149.857|61.1089|-149.857|
-135|61|O|174569.88|1995-10-21|4-NOT SPECIFIED|Clerk#000000804|0|l platelets use according t|61.1024|-149.853|61.1024|-149.853|
-160|83|O|86076.86|1996-12-19|4-NOT SPECIFIED|Clerk#000000342|0|thely special sauternes wake slyly of t|61.1891|-149.906|61.1891|-149.906|
-161|17|F|19056.99|1994-08-31|2-HIGH|Clerk#000000322|0|carefully! special instructions sin|61.1891|-149.906|61.1891|-149.906|
-162|16|O|2158.13|1995-05-08|3-MEDIUM|Clerk#000000378|0|nts hinder fluffily ironic instructions. express, express excuses |61.1891|-149.906|61.1891|-149.906|
-163|88|O|125170.86|1997-09-05|3-MEDIUM|Clerk#000000379|0|y final packages. final foxes since the quickly even|61.1891|-149.906|61.1891|-149.906|
-164|1|F|202660.52|1992-10-21|5-LOW|Clerk#000000209|0|cajole ironic courts. slyly final ideas are slyly. blithely final Tiresias sub|61.1891|-149.906|61.1891|-149.906|
-165|28|F|141824.23|1993-01-30|4-NOT SPECIFIED|Clerk#000000292|0|across the blithely regular accounts. bold|61.1891|-149.906|61.1891|-149.906|
-166|109|O|93335.60|1995-09-12|2-HIGH|Clerk#000000440|0|lets. ironic, bold asymptotes kindle|61.1891|-149.906|61.1891|-149.906|
-167|121|F|52982.23|1993-01-04|4-NOT SPECIFIED|Clerk#000000731|0|s nag furiously bold excuses. fluffily iron|61.1891|-149.906|61.1891|-149.906|
-192|83|O|133002.55|1997-11-25|5-LOW|Clerk#000000483|0|y unusual platelets among the final instructions integrate rut|61.1891|-149.906|61.1891|-149.906|
-193|80|F|48053.18|1993-08-08|1-URGENT|Clerk#000000025|0|the furiously final pin|61.1891|-149.906|61.1891|-149.906|
-194|62|F|114097.63|1992-04-05|3-MEDIUM|Clerk#000000352|0|egular requests haggle slyly regular, regular pinto beans. asymptote|61.1891|-149.906|61.1891|-149.906|
-195|136|F|120053.52|1993-12-28|3-MEDIUM|Clerk#000000216|0|old forges are furiously sheaves. slyly fi|61.1891|-149.906|61.1891|-149.906|
-196|65|F|33248.04|1993-03-17|2-HIGH|Clerk#000000988|0|beans boost at the foxes. silent foxes|61.1891|-149.906|61.1891|-149.906|
-197|34|P|100290.07|1995-04-07|2-HIGH|Clerk#000000969|0|solve quickly about the even braids. carefully express deposits affix care|61.1891|-149.906|61.1891|-149.906|
-198|112|O|125792.83|1998-01-02|4-NOT SPECIFIED|Clerk#000000331|0|its. carefully ironic requests sleep. furiously express fox|61.1891|-149.906|61.1891|-149.906|
-199|53|O|80592.44|1996-03-07|2-HIGH|Clerk#000000489|0|g theodolites. special packag|61.1891|-149.906|61.1891|-149.906|
-224|4|F|155680.60|1994-06-18|4-NOT SPECIFIED|Clerk#000000642|0|r the quickly thin courts. carefully|61.1891|-149.906|61.1891|-149.906|
-225|34|P|165890.47|1995-05-25|1-URGENT|Clerk#000000177|0|s. blithely ironic accounts wake quickly fluffily special acc|61.1891|-149.906|61.1891|-149.906|
-226|128|F|180119.22|1993-03-10|2-HIGH|Clerk#000000756|0|s are carefully at the blithely ironic acc|61.1891|-149.906|61.1891|-149.906|
-227|10|O|46076.46|1995-11-10|5-LOW|Clerk#000000919|0| express instructions. slyly regul|61.1891|-149.906|61.1891|-149.906|
-228|46|F|2638.98|1993-02-25|1-URGENT|Clerk#000000562|0|es was slyly among the regular foxes. blithely regular dependenci|61.1891|-149.906|61.1891|-149.906|
-229|112|F|142290.77|1993-12-29|1-URGENT|Clerk#000000628|0|he fluffily even instructions. furiously i|61.1891|-149.906|61.1891|-149.906|
-230|103|F|107231.60|1993-10-27|1-URGENT|Clerk#000000520|0|odolites. carefully quick requ|61.1891|-149.906|61.1891|-149.906|
-231|91|F|141554.06|1994-09-29|2-HIGH|Clerk#000000446|0| packages haggle slyly after the carefully ironic instruct|61.1891|-149.906|61.1891|-149.906|
-256|125|F|106315.25|1993-10-19|4-NOT SPECIFIED|Clerk#000000834|0|he fluffily final ideas might are final accounts. carefully f|61.1891|-149.906|61.1891|-149.906|
-257|124|O|7102.74|1998-03-28|3-MEDIUM|Clerk#000000680|0|ts against the sly warhorses cajole slyly accounts|61.1891|-149.906|61.1891|-149.906|
-258|43|F|186669.10|1993-12-29|1-URGENT|Clerk#000000167|0|dencies. blithely quick packages cajole. ruthlessly final accounts|61.1891|-149.906|61.1891|-149.906|
-259|44|F|75661.70|1993-09-29|4-NOT SPECIFIED|Clerk#000000601|0|ages doubt blithely against the final foxes. carefully express deposits dazzle|61.1891|-149.906|61.1891|-149.906|
-260|106|O|179292.14|1996-12-10|3-MEDIUM|Clerk#000000960|0|lently regular pinto beans sleep after the slyly e|61.1891|-149.906|61.1891|-149.906|
-261|47|F|201003.12|1993-06-29|3-MEDIUM|Clerk#000000310|0|ully fluffily brave instructions. furiousl|61.1891|-149.906|61.1891|-149.906|
-262|31|O|108443.84|1995-11-25|4-NOT SPECIFIED|Clerk#000000551|0|l packages. blithely final pinto beans use carefu|61.1891|-149.906|61.1891|-149.906|
-263|118|F|79782.56|1994-05-17|2-HIGH|Clerk#000000088|0| pending instructions. blithely un|61.1891|-149.906|61.1891|-149.906|
-288|8|O|163794.53|1997-02-21|1-URGENT|Clerk#000000109|0|uriously final requests. even, final ideas det|61.1891|-149.906|61.1891|-149.906|
-289|104|O|131092.67|1997-02-10|3-MEDIUM|Clerk#000000103|0|sily. slyly special excuse|61.1891|-149.906|61.1891|-149.906|
-290|118|F|62814.89|1994-01-01|4-NOT SPECIFIED|Clerk#000000735|0|efully dogged deposits. furiou|61.1891|-149.906|61.1891|-149.906|
-291|142|F|66817.05|1994-03-13|1-URGENT|Clerk#000000923|0|dolites. carefully regular pinto beans cajol|64.8541|-147.813|64.8541|-147.813|
-292|23|F|30783.05|1992-01-13|2-HIGH|Clerk#000000193|0|g pinto beans will have to sleep f|64.8414|-147.606|64.8414|-147.606|
-293|31|F|37248.78|1992-10-02|2-HIGH|Clerk#000000629|0|re bold, ironic deposits. platelets c|64.8371|-147.746|64.8371|-147.746|
-294|52|F|30059.47|1993-07-16|3-MEDIUM|Clerk#000000499|0|kly according to the frays. final dolphins affix quickly |64.8151|-147.707|64.8151|-147.707|
-295|19|F|89345.99|1994-09-29|2-HIGH|Clerk#000000155|0| unusual pinto beans play. regular ideas haggle|64.8371|-147.746|64.8371|-147.746|
-320|1|O|39835.54|1997-11-21|2-HIGH|Clerk#000000573|0|ar foxes nag blithely|64.849|-147.813|64.849|-147.813|
-321|124|F|62251.15|1993-03-21|3-MEDIUM|Clerk#000000289|0|equests run. blithely final dependencies after the deposits wake caref|64.8425|-147.724|64.8425|-147.724|
-322|134|F|127068.89|1992-03-19|1-URGENT|Clerk#000000158|0|fully across the slyly bold packages. packages against the quickly regular i|64.8425|-147.724|64.8425|-147.724|
-323|40|F|79683.42|1994-03-26|1-URGENT|Clerk#000000959|0|arefully pending foxes sleep blithely. slyly express accoun|64.849|-147.826|64.849|-147.826|
-324|106|F|26868.85|1992-03-20|1-URGENT|Clerk#000000352|0| about the ironic, regular deposits run blithely against the excuses|64.815|-147.882|64.815|-147.882|
-325|41|F|71543.41|1993-10-17|5-LOW|Clerk#000000844|0|ly sometimes pending pa|64.8906|-147.628|64.8906|-147.628|
-326|76|O|229165.17|1995-06-04|2-HIGH|Clerk#000000466|0| requests. furiously ironic asymptotes mold carefully alongside of the blit|64.8276|-147.639|64.8276|-147.639|
-327|145|P|24468.16|1995-04-17|5-LOW|Clerk#000000992|0|ng the slyly final courts. slyly even escapades eat |64.8461|-147.813|64.8461|-147.813|
-352|107|F|16003.86|1994-03-08|2-HIGH|Clerk#000000932|0|ke slyly bold pinto beans. blithely regular accounts against the spe|64.8281|-147.812|64.8281|-147.812|
-353|2|F|179984.42|1993-12-31|5-LOW|Clerk#000000449|0| quiet ideas sleep. even instructions cajole slyly. silently spe|64.8377|-147.718|64.8377|-147.718|
-354|139|O|157062.70|1996-03-14|2-HIGH|Clerk#000000511|0|ly regular ideas wake across the slyly silent ideas. final deposits eat b|64.8417|-147.718|64.8417|-147.718|
-355|71|F|69447.25|1994-06-14|5-LOW|Clerk#000000532|0|s. sometimes regular requests cajole. regular, pending accounts a|64.8145|-147.772|64.8145|-147.772|
-356|148|F|162786.67|1994-06-30|4-NOT SPECIFIED|Clerk#000000944|0|as wake along the bold accounts. even, |64.8541|-147.813|64.8541|-147.813|
-357|61|O|98723.11|1996-10-09|2-HIGH|Clerk#000000301|0|e blithely about the express, final accounts. quickl|64.8169|-147.779|64.8169|-147.779|
-358|4|F|226806.66|1993-09-20|2-HIGH|Clerk#000000392|0|l, silent instructions are slyly. silently even de|64.8378|-147.71|64.8378|-147.71|
-359|79|F|142891.22|1994-12-19|3-MEDIUM|Clerk#000000934|0|n dolphins. special courts above the carefully ironic requests use|64.8436|-147.722|64.8436|-147.722|
-384|115|F|122785.82|1992-03-03|5-LOW|Clerk#000000206|0|, even accounts use furiously packages. slyly ironic pla|64.9401|-147.402|64.9401|-147.402|
-385|34|O|50724.06|1996-03-22|5-LOW|Clerk#000000600|0|hless accounts unwind bold pain|64.8426|-147.719|64.8426|-147.719|
-386|61|F|90380.40|1995-01-25|2-HIGH|Clerk#000000648|0| haggle quickly. stealthily bold asymptotes haggle among the furiously even re|64.8534|-147.811|64.8534|-147.811|
-387|4|O|130647.18|1997-01-26|4-NOT SPECIFIED|Clerk#000000768|0| are carefully among the quickly even deposits. furiously silent req|64.9341|-147.928|64.9341|-147.928|
-388|46|F|120533.46|1992-12-16|4-NOT SPECIFIED|Clerk#000000356|0|ar foxes above the furiously ironic deposits nag slyly final reque|64.8393|-147.72|64.8393|-147.72|
-389|127|F|1984.14|1994-02-17|2-HIGH|Clerk#000000062|0|ing to the regular asymptotes. final, pending foxes about the blithely sil|64.8406|-147.731|64.8406|-147.731|
-390|103|O|168562.27|1998-04-07|5-LOW|Clerk#000000404|0|xpress asymptotes use among the regular, final pinto b|64.9281|-147.865|64.9281|-147.865|
-391|112|F|13282.23|1994-11-17|2-HIGH|Clerk#000000256|0|orges thrash fluffil|64.8371|-147.716|64.8371|-147.716|
-416|41|F|71362.50|1993-09-27|5-LOW|Clerk#000000294|0| the accounts. fluffily bold depo|64.9414|-147.841|64.9414|-147.841|
-417|55|F|91982.29|1994-02-06|3-MEDIUM|Clerk#000000468|0|ironic, even packages. thinly unusual accounts sleep along the slyly unusual |64.8363|-147.79|64.8363|-147.79|
-418|95|P|33124.96|1995-04-13|4-NOT SPECIFIED|Clerk#000000643|0|. furiously ironic instruc|64.8371|-147.716|64.8371|-147.716|
-419|118|O|111597.96|1996-10-01|3-MEDIUM|Clerk#000000376|0|osits. blithely pending theodolites boost carefully|64.8591|-147.917|64.8591|-147.917|
-420|91|O|198039.23|1995-10-31|4-NOT SPECIFIED|Clerk#000000756|0|leep carefully final excuses. fluffily pending requests unwind carefully above|64.8363|-147.79|64.8363|-147.79|
-421|40|F|1084.38|1992-02-22|5-LOW|Clerk#000000405|0|egular, even packages according to the final, un|55.3801|-131.682|55.3801|-131.682|
-422|74|O|106045.89|1997-05-31|4-NOT SPECIFIED|Clerk#000000049|0|aggle carefully across the accounts. regular accounts eat fluffi|55.3073|-131.528|55.3073|-131.528|
-423|104|O|26981.31|1996-06-01|1-URGENT|Clerk#000000674|0|quests. deposits cajole quickly. furiously bold accounts haggle q|55.3801|-131.682|55.3801|-131.682|
-448|149|O|114978.03|1995-08-21|3-MEDIUM|Clerk#000000597|0| regular, express foxes use blithely. quic|55.3601|-131.681|55.3601|-131.681|
-449|97|O|41605.63|1995-07-20|2-HIGH|Clerk#000000841|0|. furiously regular theodolites affix blithely |55.3279|-131.613|55.3279|-131.613|
-450|49|P|153386.61|1995-03-05|4-NOT SPECIFIED|Clerk#000000293|0|d theodolites. boldly bold foxes since the pack|55.3129|-131.588|55.3129|-131.588|
-451|100|O|104664.40|1998-05-25|5-LOW|Clerk#000000048|0|nic pinto beans. theodolites poach carefully; |55.3801|-131.682|55.3801|-131.682|
-452|61|O|2007.48|1997-10-14|1-URGENT|Clerk#000000498|0|t, unusual instructions above the blithely bold pint|55.3801|-131.682|55.3801|-131.682|
-453|46|O|216826.73|1997-05-26|5-LOW|Clerk#000000504|0|ss foxes. furiously regular ideas sleep according to t|55.4299|-131.789|55.4299|-131.789|
-454|49|O|23198.24|1995-12-27|5-LOW|Clerk#000000890|0|dolites sleep carefully blithely regular deposits. quickly regul|55.3801|-131.682|55.3801|-131.682|
-455|13|O|138010.76|1996-12-04|1-URGENT|Clerk#000000796|0| about the final platelets. dependen|55.3507|-131.671|55.3507|-131.671|
-480|73|F|20530.97|1993-05-08|5-LOW|Clerk#000000004|0|ealthy pinto beans. fluffily regular requests along the special sheaves wake |55.3801|-131.682|55.3801|-131.682|
-481|31|F|117827.18|1992-10-08|2-HIGH|Clerk#000000230|0|ly final ideas. packages haggle fluffily|55.3394|-131.636|55.3394|-131.636|
-482|127|O|136634.34|1996-03-26|1-URGENT|Clerk#000000295|0|ts. deposits wake: final acco|55.3801|-131.682|55.3801|-131.682|
-483|35|O|39793.05|1995-07-11|2-HIGH|Clerk#000000025|0|cross the carefully final e|55.3103|-131.582|55.3103|-131.582|
-484|55|O|219920.62|1997-01-03|3-MEDIUM|Clerk#000000545|0|grouches use. furiously bold accounts maintain. bold, regular deposits|55.3801|-131.682|55.3801|-131.682|
-485|101|O|110432.76|1997-03-26|2-HIGH|Clerk#000000105|0| regular ideas nag thinly furiously s|55.3801|-131.682|55.3801|-131.682|
-486|52|O|185968.15|1996-03-11|4-NOT SPECIFIED|Clerk#000000803|0|riously dolphins. fluffily ironic requ|55.3801|-131.682|55.3801|-131.682|
-487|109|F|48502.79|1992-08-18|1-URGENT|Clerk#000000086|0|ithely unusual courts eat accordi|55.3801|-131.682|55.3801|-131.682|
-512|64|P|124661.48|1995-05-20|5-LOW|Clerk#000000814|0|ding requests. carefully express theodolites was quickly. furious|55.3801|-131.682|55.3801|-131.682|
-513|61|O|63703.92|1995-05-01|2-HIGH|Clerk#000000522|0|regular packages. pinto beans cajole carefully against the even|55.3424|-131.634|55.3424|-131.634|
-514|76|O|104585.77|1996-04-04|2-HIGH|Clerk#000000094|0| cajole furiously. slyly final excuses cajole. slyly special instructions |55.4097|-131.729|55.4097|-131.729|
-515|142|F|153720.22|1993-08-29|4-NOT SPECIFIED|Clerk#000000700|0|eposits are furiously furiously silent pinto beans. pending pack|55.3801|-131.682|55.3801|-131.682|
-516|44|O|10677.86|1998-04-21|2-HIGH|Clerk#000000305|0|lar, unusual platelets are carefully. even courts sleep bold, final pinto bea|55.3801|-131.682|55.3801|-131.682|
-517|10|O|82197.79|1997-04-07|5-LOW|Clerk#000000359|0|slyly pending deposits cajole quickly packages. furiou|55.3462|-131.658|55.3462|-131.658|
-518|145|O|223537.09|1998-02-08|2-HIGH|Clerk#000000768|0| the carefully bold accounts. quickly regular excuses are|55.3801|-131.682|55.3801|-131.682|
-519|64|O|95731.50|1997-10-31|1-URGENT|Clerk#000000985|0|ains doze furiously against the f|55.3801|-131.682|55.3801|-131.682|
-544|94|F|47627.89|1993-02-17|2-HIGH|Clerk#000000145|0|the special, final accounts. dogged dolphins|55.3801|-131.682|55.3801|-131.682|
-545|64|O|23476.12|1995-11-07|2-HIGH|Clerk#000000537|0|as. blithely final hockey players about th|55.3801|-131.682|55.3801|-131.682|
-546|145|O|14790.37|1996-11-01|2-HIGH|Clerk#000000041|0|osits sleep. slyly special dolphins about the q|55.3801|-131.682|55.3801|-131.682|
-547|100|O|96855.29|1996-06-22|3-MEDIUM|Clerk#000000976|0|ing accounts eat. carefully regular packa|55.3801|-131.682|55.3801|-131.682|
-548|124|F|99088.75|1994-09-21|1-URGENT|Clerk#000000435|0|arefully express instru|55.3801|-131.682|55.3801|-131.682|
-549|110|F|141679.41|1992-07-13|1-URGENT|Clerk#000000196|0|ideas alongside of |55.3801|-131.682|55.3801|-131.682|
-550|25|O|33123.28|1995-08-02|1-URGENT|Clerk#000000204|0|t requests. blithely |61.5856|-149.316|61.5856|-149.316|
-551|91|O|46355.83|1995-05-30|1-URGENT|Clerk#000000179|0|xpress accounts boost quic|61.5781|-149.429|61.5781|-149.429|
-576|31|O|18307.45|1997-05-13|3-MEDIUM|Clerk#000000955|0|l requests affix regular requests. final account|61.6141|-149.457|61.6141|-149.457|
-577|56|F|34768.68|1994-12-19|5-LOW|Clerk#000000154|0| deposits engage stealthil|61.5801|-149.461|61.5801|-149.461|
-578|94|O|70392.02|1997-01-10|5-LOW|Clerk#000000281|0|e blithely even packages. slyly pending platelets bes|61.9071|-150.067|61.9071|-150.067|
-579|68|O|120828.12|1998-03-11|2-HIGH|Clerk#000000862|0| regular instructions. blithely even p|61.5928|-149.392|61.5928|-149.392|
-580|61|O|88219.12|1997-07-05|2-HIGH|Clerk#000000314|0|tegrate fluffily regular accou|61.6141|-149.457|61.6141|-149.457|
-581|70|O|126066.00|1997-02-23|4-NOT SPECIFIED|Clerk#000000239|0| requests. even requests use slyly. blithely ironic |61.5792|-149.36|61.5792|-149.36|
-582|50|O|129004.81|1997-10-21|1-URGENT|Clerk#000000378|0|n pinto beans print a|61.6049|-149.463|61.6049|-149.463|
-583|49|O|127817.38|1997-03-19|3-MEDIUM|Clerk#000000792|0|efully express requests. a|61.6099|-149.328|61.6099|-149.328|
-608|26|O|62567.99|1996-02-28|3-MEDIUM|Clerk#000000995|0|nic waters wake slyly slyly expre|61.5531|-149.651|61.5531|-149.651|
-609|127|F|21088.59|1994-06-01|3-MEDIUM|Clerk#000000348|0|- ironic gifts believe furiously ca|61.6141|-149.457|61.6141|-149.457|
-610|52|O|175142.28|1995-08-02|1-URGENT|Clerk#000000610|0|totes. ironic, unusual packag|61.6141|-149.457|61.6141|-149.457|
-611|106|F|73907.63|1993-01-27|1-URGENT|Clerk#000000401|0|ounts detect furiously ac|61.5531|-149.651|61.5531|-149.651|
-612|82|F|145695.42|1992-10-21|3-MEDIUM|Clerk#000000759|0|boost quickly quickly final excuses. final foxes use bravely afte|61.6141|-149.457|61.6141|-149.457|
-613|139|O|33396.35|1995-06-18|2-HIGH|Clerk#000000172|0|ts hinder among the deposits. fluffily ironic depos|61.7321|-150.12|61.7321|-150.12|
-614|134|F|218116.21|1992-12-01|2-HIGH|Clerk#000000388|0| deposits! even, daring theodol|61.6141|-149.457|61.6141|-149.457|
-615|67|F|32890.89|1992-05-09|5-LOW|Clerk#000000388|0|t to promise asymptotes. packages haggle alongside of the fluffil|61.582|-149.441|61.582|-149.441|
-640|97|F|145495.62|1993-01-23|2-HIGH|Clerk#000000433|0|r, unusual accounts boost carefully final ideas. slyly silent theod|61.5818|-149.44|61.5818|-149.44|
-641|133|F|120626.49|1993-08-30|5-LOW|Clerk#000000175|0|ents cajole furiously about the quickly silent pac|61.6141|-149.457|61.6141|-149.457|
-642|40|F|22994.51|1993-12-16|3-MEDIUM|Clerk#000000357|0| among the requests wake slyly alongside of th|61.7321|-150.12|61.7321|-150.12|
-643|58|P|180396.95|1995-03-25|2-HIGH|Clerk#000000354|0|g dependencies. regular accounts |61.6308|-149.415|61.6308|-149.415|
-644|8|F|201268.06|1992-05-01|1-URGENT|Clerk#000000550|0| blithely unusual platelets haggle ironic, special excuses. excuses unwi|61.5801|-149.461|61.5801|-149.461|
-645|115|F|234763.73|1994-12-03|2-HIGH|Clerk#000000090|0|quickly daring theodolites across the regu|61.5811|-149.444|61.5811|-149.444|
-646|52|F|142070.65|1994-11-22|2-HIGH|Clerk#000000203|0|carefully even foxes. fina|61.6521|-149.92|61.6521|-149.92|
-647|143|O|56449.23|1997-08-07|1-URGENT|Clerk#000000270|0|egular pearls. carefully express asymptotes are. even account|61.6141|-149.307|61.6141|-149.307|
-672|109|F|89877.09|1994-04-14|5-LOW|Clerk#000000106|0|egular requests are furiously according to |61.6168|-149.328|61.6168|-149.328|
-673|80|F|21137.08|1994-03-10|1-URGENT|Clerk#000000448|0| special pinto beans use quickly furiously even depende|61.5714|-149.381|61.5714|-149.381|
-674|34|F|27204.60|1992-08-29|5-LOW|Clerk#000000448|0|ully special deposits. furiously final warhorses affix carefully. fluffily f|61.6521|-149.92|61.6521|-149.92|
-675|13|O|125188.72|1997-07-31|2-HIGH|Clerk#000000168|0|ffily between the careful|61.5858|-149.376|61.5858|-149.376|
-676|38|O|163966.67|1996-12-13|2-HIGH|Clerk#000000248|0|the final deposits. special, pending|61.5822|-149.463|61.5822|-149.463|
-677|124|F|147915.68|1993-11-24|3-MEDIUM|Clerk#000000824|0|uriously special pinto beans cajole carefully. fi|61.5861|-149.303|61.5861|-149.303|
-678|131|F|135761.05|1993-02-27|5-LOW|Clerk#000000530|0|. blithely final somas about the|61.5821|-149.438|61.5821|-149.438|
-679|49|O|8945.03|1995-12-15|2-HIGH|Clerk#000000853|0|tealthy, final pinto beans haggle slyly. pending platelets about the special, |61.6281|-149.338|61.6281|-149.338|
-704|85|O|56210.26|1996-11-21|3-MEDIUM|Clerk#000000682|0|blithely pending platelets wake alongside of the final, iron|61.5771|-149.335|61.5771|-149.335|
-705|43|O|83773.49|1997-02-13|4-NOT SPECIFIED|Clerk#000000294|0|ithely regular dependencies. express, even packages sleep slyly pending t|61.5917|-149.464|61.5917|-149.464|
-706|148|O|23973.60|1995-09-09|1-URGENT|Clerk#000000448|0|g the packages. deposits caj|61.1927|-149.86|61.1927|-149.86|
-707|118|F|58218.35|1994-11-20|3-MEDIUM|Clerk#000000199|0| ideas about the silent, bold deposits nag dolphins|61.1879|-149.85|61.1879|-149.85|
-708|32|O|100445.59|1998-07-03|3-MEDIUM|Clerk#000000101|0|lphins cajole about t|61.1814|-149.849|61.1814|-149.849|
-709|37|O|72055.87|1998-04-21|1-URGENT|Clerk#000000461|0|ons alongside of the carefully bold pinto bea|61.2104|-149.892|61.2104|-149.892|
-710|133|F|208974.42|1993-01-02|5-LOW|Clerk#000000026|0| regular, regular requests boost. fluffily re|61.2093|-149.903|61.2093|-149.903|
-711|64|F|92484.70|1993-09-23|4-NOT SPECIFIED|Clerk#000000856|0|its. fluffily regular gifts are furi|61.1481|-149.829|61.1481|-149.829|
-736|47|O|130204.17|1998-06-21|5-LOW|Clerk#000000881|0|refully of the final pi|61.2161|-149.876|61.2161|-149.876|
-737|121|F|12984.85|1992-04-26|5-LOW|Clerk#000000233|0|ake blithely express, ironic theodolites. blithely special accounts wa|61.1972|-149.75|61.1972|-149.75|
-738|22|F|114145.18|1993-03-02|4-NOT SPECIFIED|Clerk#000000669|0|ly even foxes. furiously regular accounts cajole ca|61.2066|-149.887|61.2066|-149.887|
-739|1|O|159171.69|1998-05-31|5-LOW|Clerk#000000900|0| against the slyly ironic packages nag slyly ironic|61.2161|-149.876|61.2161|-149.876|
-740|44|O|83490.99|1995-07-16|3-MEDIUM|Clerk#000000583|0|courts haggle furiously across the final, regul|61.195|-149.834|61.195|-149.834|
-741|106|O|47985.98|1998-07-07|2-HIGH|Clerk#000000295|0|ic instructions. slyly express instructions solv|61.2038|-149.808|61.2038|-149.808|
-742|103|F|207632.55|1994-12-23|5-LOW|Clerk#000000543|0|equests? slyly ironic dolphins boost carefully above the blithely|61.1228|-149.862|61.1228|-149.862|
-743|79|O|23614.89|1996-10-04|4-NOT SPECIFIED|Clerk#000000933|0|eans. furiously ironic deposits sleep carefully carefully qui|61.2005|-149.785|61.2005|-149.785|
-768|98|O|220636.82|1996-08-20|3-MEDIUM|Clerk#000000411|0|jole slyly ironic packages. slyly even idea|61.181|-149.825|61.181|-149.825|
-769|80|F|43092.76|1993-06-02|3-MEDIUM|Clerk#000000172|0|ggle furiously. ironic packages haggle slyly. bold platelets affix s|61.1867|-149.919|61.1867|-149.919|
-770|32|O|64271.75|1998-05-23|5-LOW|Clerk#000000572|0|heodolites. furiously special pinto beans cajole pac|61.1955|-149.911|61.1955|-149.911|
-771|46|O|105302.05|1995-06-17|1-URGENT|Clerk#000000105|0|s. furiously final instructions across the deposit|61.1089|-149.858|61.1089|-149.858|
-772|97|F|128234.96|1993-04-17|2-HIGH|Clerk#000000430|0|s boost blithely fluffily idle ideas? fluffily even pin|61.1805|-149.889|61.1805|-149.889|
-773|133|F|146862.27|1993-09-26|3-MEDIUM|Clerk#000000307|0|tions are quickly accounts. accounts use bold, even pinto beans. gifts ag|61.1534|-149.985|61.1534|-149.985|
-774|80|O|145857.60|1995-12-04|1-URGENT|Clerk#000000883|0|tealthily even depths|61.1901|-149.911|61.1901|-149.911|
-775|134|F|59455.61|1995-03-18|1-URGENT|Clerk#000000191|0|kly express requests. fluffily silent accounts poach furiously|61.2122|-149.734|61.2122|-149.734|
-800|56|O|87892.38|1998-07-14|2-HIGH|Clerk#000000213|0|y alongside of the pending packages? final platelets nag fluffily carefu|61.1951|-149.906|61.1951|-149.906|
-801|118|F|127717.72|1992-02-18|1-URGENT|Clerk#000000186|0|iously from the furiously enticing reques|61.2043|-149.869|61.2043|-149.869|
-802|137|F|156381.95|1995-01-05|1-URGENT|Clerk#000000516|0|posits. ironic, pending requests cajole. even theodol|61.2036|-149.869|61.2036|-149.869|
-803|16|O|27629.66|1997-04-29|5-LOW|Clerk#000000260|0|ic instructions. even deposits haggle furiously at the deposits-- regular de|61.1883|-149.886|61.1883|-149.886|
-804|50|F|94400.43|1993-03-12|3-MEDIUM|Clerk#000000931|0|s. blithely final foxes are about the packag|61.2141|-149.864|61.2141|-149.864|
-805|127|O|90042.41|1995-07-05|4-NOT SPECIFIED|Clerk#000000856|0|y according to the fluffily |61.1955|-149.782|61.1955|-149.782|
-806|131|O|26839.16|1996-06-20|2-HIGH|Clerk#000000240|0| the ironic packages wake carefully fina|61.2183|-149.894|61.2183|-149.894|
-807|145|F|222392.53|1993-11-24|3-MEDIUM|Clerk#000000012|0|refully special tithes. blithely regular accoun|61.1417|-149.864|61.1417|-149.864|
-832|29|F|68494.08|1992-04-19|5-LOW|Clerk#000000495|0|xes. bravely regular packages sleep up the furiously bold accou|61.1883|-149.883|61.1883|-149.883|
-833|56|F|49033.69|1994-02-13|3-MEDIUM|Clerk#000000437|0|ts haggle quickly across the slyl|61.2161|-149.876|61.2161|-149.876|
-834|43|F|46459.92|1994-05-23|3-MEDIUM|Clerk#000000805|0| sleep. quickly even foxes are boldly. slyly express requests use slyly|61.2193|-149.869|61.2193|-149.869|
-835|65|O|62430.67|1995-10-08|4-NOT SPECIFIED|Clerk#000000416|0|s about the carefully special foxes haggle quickly about the|61.2191|-149.888|61.2191|-149.888|
-836|70|O|72843.48|1996-11-25|4-NOT SPECIFIED|Clerk#000000729|0|ely bold excuses sleep regular ideas. furiously unusual ideas wake furiou|61.2191|-149.888|61.2191|-149.888|
-837|116|F|60918.41|1994-06-15|4-NOT SPECIFIED|Clerk#000000563|0|kages sleep slyly above the ironic, final orbits|61.2191|-149.888|61.2191|-149.888|
-838|17|O|82918.36|1998-01-29|5-LOW|Clerk#000000213|0| slyly around the slyly even|61.2191|-149.888|61.2191|-149.888|
-839|28|O|70182.63|1995-08-08|1-URGENT|Clerk#000000951|0|the carefully even platelets. furiously unusual fo|61.2191|-149.888|61.2191|-149.888|
-864|139|O|74710.74|1997-08-17|1-URGENT|Clerk#000000036|0|ly after the slyly regular deposits. express, regular asymptotes nag ca|61.2191|-149.888|61.2191|-149.888|
-865|4|F|70430.54|1993-05-04|3-MEDIUM|Clerk#000000337|0|. special packages wake after the carefully final accounts. express pinto be|61.2191|-149.888|61.2191|-149.888|
-866|40|F|4766.19|1992-11-28|3-MEDIUM|Clerk#000000718|0|ins after the even, even accounts nod blithel|61.2191|-149.888|61.2191|-149.888|
-867|26|F|7471.75|1993-11-16|3-MEDIUM|Clerk#000000877|0|pades nag quickly final, |61.2191|-149.888|61.2191|-149.888|
-868|104|F|127345.45|1992-06-09|4-NOT SPECIFIED|Clerk#000000782|0|onic theodolites print carefully. blithely dogge|61.2191|-149.888|61.2191|-149.888|
-869|136|O|58932.19|1997-01-12|2-HIGH|Clerk#000000245|0|ar sheaves are slowly. slyly even attainments boost theodolites. furiously|61.2191|-149.888|61.2191|-149.888|
-870|34|F|40492.37|1993-06-20|4-NOT SPECIFIED|Clerk#000000123|0|blithely ironic ideas nod. sly, r|61.2191|-149.888|61.2191|-149.888|
-871|16|O|172861.58|1995-11-15|5-LOW|Clerk#000000882|0|oss the ironic theodolites.|61.1891|-149.906|61.1891|-149.906|
-896|2|F|169847.63|1993-03-09|1-URGENT|Clerk#000000187|0|inal packages eat blithely according to the warhorses. furiously quiet de|61.2191|-149.888|61.2191|-149.888|
-897|49|P|57697.44|1995-03-20|1-URGENT|Clerk#000000316|0| wake quickly against |61.2191|-149.888|61.2191|-149.888|
-898|55|F|101020.75|1993-06-03|2-HIGH|Clerk#000000611|0|. unusual pinto beans haggle quickly across |61.1101|-149.857|61.1101|-149.857|
-899|109|O|125562.09|1998-04-08|5-LOW|Clerk#000000575|0|rts engage carefully final theodolites.|61.1101|-149.857|61.1101|-149.857|
-900|46|F|120073.51|1994-10-01|4-NOT SPECIFIED|Clerk#000000060|0| fluffily express deposits nag furiousl|61.1101|-149.857|61.1101|-149.857|
-901|13|O|81826.12|1998-07-21|4-NOT SPECIFIED|Clerk#000000929|0|lyly even foxes are furious, silent requests. requests about the quickly |61.1101|-149.857|61.1101|-149.857|
-902|10|F|37348.62|1994-07-27|4-NOT SPECIFIED|Clerk#000000811|0|yly final requests over the furiously regula|61.1101|-149.857|61.1101|-149.857|
-903|11|O|109351.87|1995-07-07|4-NOT SPECIFIED|Clerk#000000793|0|e slyly about the final pl|61.1101|-149.857|61.1101|-149.857|
-928|67|F|228136.49|1995-03-02|5-LOW|Clerk#000000450|0|ithely express pinto beans. |61.1101|-149.857|61.1101|-149.857|
-929|83|F|109301.02|1992-10-02|2-HIGH|Clerk#000000160|0|its. furiously even foxes affix carefully finally silent accounts. express req|61.1101|-149.857|61.1101|-149.857|
-930|131|F|199102.23|1994-12-17|1-URGENT|Clerk#000000004|0| accounts nag slyly. ironic, ironic accounts wake blithel|61.1101|-149.857|61.1101|-149.857|
-931|103|F|117909.23|1992-12-07|1-URGENT|Clerk#000000881|0|ss packages haggle furiously express, regular deposits. even, e|61.1101|-149.857|61.1101|-149.857|
-932|41|O|40234.50|1997-05-16|2-HIGH|Clerk#000000218|0|ly express instructions boost furiously reg|61.1101|-149.857|61.1101|-149.857|
-933|97|F|71349.30|1992-08-05|4-NOT SPECIFIED|Clerk#000000752|0|ial courts wake permanently against the furiously regular ideas. unusual |61.1101|-149.857|61.1101|-149.857|
-934|52|O|17213.59|1996-07-03|1-URGENT|Clerk#000000229|0|ts integrate carefully. sly, regular deposits af|61.1101|-149.857|61.1101|-149.857|
-935|50|O|97733.87|1997-09-24|5-LOW|Clerk#000000180|0|iously final deposits cajole. blithely even packages |61.1101|-149.857|61.1101|-149.857|
-960|35|F|63537.13|1994-09-21|3-MEDIUM|Clerk#000000120|0|regular accounts. requests|61.1101|-149.857|61.1101|-149.857|
-961|56|P|158893.16|1995-06-04|4-NOT SPECIFIED|Clerk#000000720|0|ons nag furiously among the quickl|61.1101|-149.857|61.1101|-149.857|
-962|37|F|98258.73|1994-05-06|5-LOW|Clerk#000000463|0|ments nag deposits. fluffily ironic a|61.1101|-149.857|61.1101|-149.857|
-963|26|F|53287.25|1994-05-26|3-MEDIUM|Clerk#000000497|0|uses haggle carefully. slyly even dependencies after the packages ha|61.1101|-149.857|61.1101|-149.857|
-964|76|O|131146.47|1995-05-20|3-MEDIUM|Clerk#000000657|0|print blithely ironic, careful theodolit|61.1101|-149.857|61.1101|-149.857|
-965|70|P|41758.44|1995-05-15|5-LOW|Clerk#000000218|0|iously special packages. slyly pending requests are carefully |64.8591|-147.917|64.8591|-147.917|
-966|14|O|120516.93|1998-04-30|2-HIGH|Clerk#000000239|0|special deposits. furious|64.8273|-147.715|64.8273|-147.715|
-967|110|F|179287.95|1992-06-21|3-MEDIUM|Clerk#000000167|0|excuses engage quickly bold dep|64.8281|-147.715|64.8281|-147.715|
-992|55|O|133665.12|1997-11-11|3-MEDIUM|Clerk#000000875|0|ts. regular pinto beans thrash carefully sl|64.8552|-147.763|64.8552|-147.763|
-993|80|O|198238.65|1995-09-10|3-MEDIUM|Clerk#000000894|0|quickly express accounts among the furiously bol|64.8481|-147.684|64.8481|-147.684|
-994|2|F|41433.48|1994-04-20|5-LOW|Clerk#000000497|0|ole. slyly bold excuses nag caref|64.8522|-147.773|64.8522|-147.773|
-995|116|P|135157.92|1995-05-31|3-MEDIUM|Clerk#000000439|0|deas. blithely final deposits play. express accounts wake blithely caref|64.8467|-147.703|64.8467|-147.703|
-996|71|O|47447.63|1997-12-29|1-URGENT|Clerk#000000497|0|arefully final packages into the slyly final requests affix blit|64.8963|-147.662|64.8963|-147.662|
-997|109|O|27561.82|1997-05-19|2-HIGH|Clerk#000000651|0|ly express depths. furiously final requests haggle furiously. carefu|64.8372|-147.796|64.8372|-147.796|
-998|32|F|65269.38|1994-11-26|4-NOT SPECIFIED|Clerk#000000956|0|ronic dolphins. ironic, bold ideas haggle furiously furious|64.8312|-147.716|64.8312|-147.716|
-999|61|F|145249.13|1993-09-05|5-LOW|Clerk#000000464|0|pitaphs sleep. regular accounts use. f|64.811|-147.71|64.811|-147.71|
-1024|4|O|176084.63|1997-12-23|5-LOW|Clerk#000000903|0| blithely. even, express theodolites cajole slyly across|64.8971|-147.663|64.8971|-147.663|
-1025|103|F|82034.03|1995-05-05|2-HIGH|Clerk#000000376|0|ross the slyly final pa|64.85|-147.699|64.85|-147.699|
-1026|73|O|36464.76|1997-06-04|5-LOW|Clerk#000000223|0|s wake blithely. special acco|64.8389|-147.743|64.8389|-147.743|
-1027|128|F|112770.89|1992-06-03|3-MEDIUM|Clerk#000000241|0|equests cajole. slyly final pinto bean|64.781|-148|64.781|-148|
-1028|70|F|153864.67|1994-01-01|2-HIGH|Clerk#000000131|0|ts are. final, silent deposits are among the fl|64.8377|-147.718|64.8377|-147.718|
-1029|130|F|47440.91|1994-06-21|2-HIGH|Clerk#000000700|0|quests sleep. slyly even foxes wake quickly final theodolites. clo|64.8248|-147.886|64.8248|-147.886|
-1030|134|F|16346.94|1994-06-15|5-LOW|Clerk#000000422|0|ully ironic accounts sleep carefully. requests are carefully alongside of the |64.818|-147.679|64.818|-147.679|
-1031|4|F|128024.71|1994-09-01|3-MEDIUM|Clerk#000000448|0|s; ironic theodolites along the carefully ex|64.8271|-147.79|64.8271|-147.79|
-1056|28|F|38446.39|1995-02-11|1-URGENT|Clerk#000000125|0|t, even deposits hang about the slyly special i|64.8451|-147.812|64.8451|-147.812|
-1057|76|F|108107.42|1992-02-20|1-URGENT|Clerk#000000124|0|cuses dazzle carefully careful, ironic pinto beans. carefully even theod|64.8311|-147.729|64.8311|-147.729|
-1058|53|F|89359.11|1993-04-26|3-MEDIUM|Clerk#000000373|0|kly pending courts haggle. blithely regular sheaves integrate carefully fi|64.8454|-147.855|64.8454|-147.855|
-1059|127|F|198360.22|1994-02-27|1-URGENT|Clerk#000000104|0|en accounts. carefully bold packages cajole daringly special depende|64.8302|-147.744|64.8302|-147.744|
-1060|140|F|121994.04|1993-02-21|3-MEDIUM|Clerk#000000989|0|l platelets sleep quickly slyly special requests. furiously |64.8113|-147.91|64.8113|-147.91|
-1061|103|O|166947.75|1998-05-15|5-LOW|Clerk#000000576|0|uests sleep at the packages. fur|64.8271|-147.789|64.8271|-147.789|
-1062|106|O|39805.04|1997-01-15|1-URGENT|Clerk#000000152|0|eposits use blithely |64.8451|-147.698|64.8451|-147.698|
-1063|37|F|41392.31|1994-04-02|2-HIGH|Clerk#000000024|0|deposits nag quickly regular deposits. quickl|64.8586|-147.69|64.8586|-147.69|
-1088|148|F|47120.41|1992-05-21|5-LOW|Clerk#000000347|0|counts are blithely. platelets print. carefully |64.8507|-147.702|64.8507|-147.702|
-1089|49|O|103192.74|1996-05-04|4-NOT SPECIFIED|Clerk#000000226|0|ns haggle ruthlessly. even requests are quickly abov|64.8371|-147.716|64.8371|-147.716|
-1090|19|O|32929.30|1997-11-15|2-HIGH|Clerk#000000300|0| furiously regular platelets haggle along the slyly unusual foxes! |64.8449|-147.743|64.8449|-147.743|
-1091|83|O|35795.22|1996-08-27|1-URGENT|Clerk#000000549|0| even pinto beans haggle quickly alongside of the eve|64.8475|-147.706|64.8475|-147.706|
-1092|124|P|85552.21|1995-03-04|3-MEDIUM|Clerk#000000006|0|re quickly along the blithe|64.8452|-147.714|64.8452|-147.714|
-1093|101|O|79189.58|1997-07-31|4-NOT SPECIFIED|Clerk#000000159|0| after the carefully ironic requests. carefully ironic packages wake fluffil|64.8125|-147.787|64.8125|-147.787|
-1094|145|O|9006.25|1997-12-24|3-MEDIUM|Clerk#000000570|0|beans affix furiously about the pending, even deposits. finally pendi|55.3801|-131.682|55.3801|-131.682|
-1095|145|O|178491.24|1995-08-22|3-MEDIUM|Clerk#000000709|0|sly bold requests cajole carefully according to|55.3801|-131.682|55.3801|-131.682|
-1120|140|O|107958.62|1997-11-07|3-MEDIUM|Clerk#000000319|0|lly special requests. slyly pending platelets are quickly pending requ|55.3801|-131.682|55.3801|-131.682|
-1121|29|O|241837.88|1997-01-13|3-MEDIUM|Clerk#000000541|0|r escapades. deposits above the fluffily bold requests hag|55.3801|-131.682|55.3801|-131.682|
-1122|121|O|179747.47|1997-01-10|1-URGENT|Clerk#000000083|0|uffily carefully final theodolites. furiously express packages affix|55.3801|-131.682|55.3801|-131.682|
-1123|73|O|93259.93|1996-08-03|3-MEDIUM|Clerk#000000929|0|uriously pending requests. slyly regular instruction|55.3801|-131.682|55.3801|-131.682|
-1124|80|O|141858.97|1998-07-30|5-LOW|Clerk#000000326|0|regular pinto beans along the fluffily silent packages|55.3599|-131.687|55.3599|-131.687|
-1125|25|F|80438.38|1994-10-27|2-HIGH|Clerk#000000510|0|ithely final requests. i|55.4381|-131.803|55.4381|-131.803|
-1126|145|O|59982.31|1998-01-28|4-NOT SPECIFIED|Clerk#000000928|0|d slyly regular ideas: special ideas believe slyly. slyly ironic sheaves w|55.3751|-131.718|55.3751|-131.718|
-1127|58|O|103320.91|1995-09-19|4-NOT SPECIFIED|Clerk#000000397|0|usly silent, regular pinto beans. blithely express requests boos|55.3421|-131.641|55.3421|-131.641|
-1152|49|F|51775.54|1994-08-14|4-NOT SPECIFIED|Clerk#000000496|0|equests. deposits ab|55.3408|-131.64|55.3408|-131.64|
-1153|121|O|220727.97|1996-04-18|5-LOW|Clerk#000000059|0| across the pending deposi|55.2978|-131.534|55.2978|-131.534|
-1154|37|F|192417.85|1992-02-15|1-URGENT|Clerk#000000268|0|old asymptotes are special requests. blithely even deposits sleep furiously|55.3801|-131.682|55.3801|-131.682|
-1155|149|O|126902.81|1997-10-06|2-HIGH|Clerk#000000164|0|c deposits haggle among the ironic, even requests. carefully ironic sheaves n|55.3801|-131.682|55.3801|-131.682|
-1156|133|O|217682.81|1996-10-19|1-URGENT|Clerk#000000200|0| blithely ironic dolphins. furiously pendi|55.3421|-131.622|55.3421|-131.622|
-1157|97|O|85394.06|1998-01-14|4-NOT SPECIFIED|Clerk#000000207|0|out the regular excuses boost carefully against the furio|55.3801|-131.682|55.3801|-131.682|
-1158|142|O|31075.51|1996-06-30|2-HIGH|Clerk#000000549|0|integrate slyly furiously ironic deposit|55.3801|-131.682|55.3801|-131.682|
-1159|70|F|55553.68|1992-09-18|3-MEDIUM|Clerk#000000992|0|ts may sleep. requests according to the|55.3801|-131.682|55.3801|-131.682|
-1184|89|O|39700.29|1997-10-26|5-LOW|Clerk#000000777|0|iously even packages haggle fluffily care|55.3267|-131.523|55.3267|-131.523|
-1185|74|F|47033.21|1992-08-24|5-LOW|Clerk#000000344|0| even escapades are. package|55.3522|-131.685|55.3522|-131.685|
-1186|59|O|82026.18|1996-08-15|4-NOT SPECIFIED|Clerk#000000798|0|ingly regular pinto beans: instructi|55.5351|-133.014|55.5351|-133.014|
-1187|134|F|85948.02|1992-11-20|3-MEDIUM|Clerk#000000047|0|s after the furiously final deposits boost slyly under the|55.5351|-133.014|55.5351|-133.014|
-1188|20|O|54655.07|1996-04-11|2-HIGH|Clerk#000000256|0|ully ironic deposits. slyl|55.5351|-133.014|55.5351|-133.014|
-1189|46|F|71017.99|1994-04-09|1-URGENT|Clerk#000000243|0|f the even accounts. courts print blithely ironic accounts. sile|55.5351|-133.014|55.5351|-133.014|
-1190|13|O|31043.39|1997-03-16|5-LOW|Clerk#000000575|0|ccounts above the foxes integrate carefully after the |55.5351|-133.014|55.5351|-133.014|
-1191|112|O|28623.04|1995-11-07|3-MEDIUM|Clerk#000000011|0|uests nag furiously. carefully even requests|55.4691|-132.855|55.4691|-132.855|
-1216|122|F|68056.57|1992-12-07|5-LOW|Clerk#000000918|0|nal foxes around the e|55.5511|-133.081|55.5511|-133.081|
-1217|7|F|40982.08|1992-04-26|4-NOT SPECIFIED|Clerk#000000538|0| foxes nag quickly. ironic excuses nod. blithely pending|55.5351|-133.014|55.5351|-133.014|
-1218|10|F|99834.47|1994-06-20|4-NOT SPECIFIED|Clerk#000000994|0|s cajole. special, silent deposits about the theo|55.5531|-133.097|55.5531|-133.097|
-1219|28|O|10163.56|1995-10-05|3-MEDIUM|Clerk#000000800|0|od carefully. slyly final dependencies across the even fray|55.5351|-133.014|55.5351|-133.014|
-1220|49|O|122157.14|1996-08-29|1-URGENT|Clerk#000000712|0|inal theodolites wake. fluffily ironic asymptotes cajol|55.4726|-131.793|55.4726|-131.793|
-1221|14|F|117397.16|1992-04-19|4-NOT SPECIFIED|Clerk#000000852|0| detect against the silent, even deposits. carefully ironic|55.3801|-131.682|55.3801|-131.682|
-1222|10|F|47623.94|1993-02-05|3-MEDIUM|Clerk#000000811|0|theodolites use quickly even accounts. carefully final asympto|55.3801|-131.682|55.3801|-131.682|
-1223|10|O|26714.67|1996-05-25|4-NOT SPECIFIED|Clerk#000000238|0|posits was blithely fr|55.3801|-131.682|55.3801|-131.682|
-1248|49|F|210713.88|1992-01-02|1-URGENT|Clerk#000000890|0|t the carefully regular dugouts. s|61.5745|-149.562|61.5745|-149.562|
-1249|149|F|45889.09|1994-01-05|1-URGENT|Clerk#000000095|0|al ideas sleep above the pending pin|61.7321|-150.12|61.7321|-150.12|
-1250|37|F|12907.62|1992-09-29|4-NOT SPECIFIED|Clerk#000000652|0|ts after the fluffily pending instructions use slyly about the s|61.5421|-149.419|61.5421|-149.419|
-1251|38|O|109536.55|1997-10-30|1-URGENT|Clerk#000000276|0|, brave sauternes. deposits boost fluffily.|61.5722|-149.702|61.5722|-149.702|
-1252|149|O|93403.05|1997-08-04|5-LOW|Clerk#000000348|0|ng the slyly regular excuses. special courts nag furiously blithely e|61.5743|-149.405|61.5743|-149.405|
-1253|115|F|92730.74|1993-01-26|1-URGENT|Clerk#000000775|0| requests sleep furiously even foxes. ruthless packag|61.578|-149.441|61.578|-149.441|
-1254|70|O|94649.25|1995-12-22|1-URGENT|Clerk#000000607|0| pinto beans. carefully regular request|61.5826|-149.427|61.5826|-149.427|
-1255|122|F|62518.31|1994-05-30|4-NOT SPECIFIED|Clerk#000000798|0|ct slyly regular accounts. quick|61.5586|-149.351|61.5586|-149.351|
-1280|97|F|91664.85|1993-01-11|5-LOW|Clerk#000000160|0|posits thrash quickly after the theodolites. furiously iro|61.5844|-149.442|61.5844|-149.442|
-1281|62|F|165454.51|1994-12-11|1-URGENT|Clerk#000000430|0|counts. carefully pending accounts eat |61.5817|-149.472|61.5817|-149.472|
-1282|116|F|61297.42|1992-02-29|4-NOT SPECIFIED|Clerk#000000168|0|he quickly special packages. furiously final re|61.6141|-149.457|61.6141|-149.457|
-1283|118|O|202623.92|1996-08-30|4-NOT SPECIFIED|Clerk#000000260|0| pinto beans boost slyly ac|61.5761|-149.602|61.5761|-149.602|
-1284|134|O|106122.38|1996-01-07|2-HIGH|Clerk#000000492|0|s. blithely silent deposits s|61.6141|-149.457|61.6141|-149.457|
-1285|11|F|139124.72|1992-06-01|1-URGENT|Clerk#000000423|0|cial deposits cajole after the ironic requests. p|61.58|-149.434|61.58|-149.434|
-1286|109|F|207291.83|1993-05-14|4-NOT SPECIFIED|Clerk#000000939|0| deposits use carefully from the excuses. slyly bold p|61.6002|-149.429|61.6002|-149.429|
-1287|19|F|131432.42|1994-07-05|2-HIGH|Clerk#000000288|0|ly ironic dolphins integrate furiously among the final packages. st|61.569|-149.347|61.569|-149.347|
-1312|112|F|58111.00|1994-05-19|3-MEDIUM|Clerk#000000538|0|n, express accounts across the ironic|61.5812|-149.448|61.5812|-149.448|
-1313|148|F|46598.65|1994-09-13|1-URGENT|Clerk#000000774|0|ld accounts. regular deposits cajole. ironically pending theodolites use car|61.6141|-149.457|61.6141|-149.457|
-1314|143|F|56207.66|1994-05-13|3-MEDIUM|Clerk#000000485|0|ickly blithe packages nod ideas. furiously bold braids boost around the car|61.6141|-149.457|61.6141|-149.457|
-1315|22|O|121935.23|1998-03-22|5-LOW|Clerk#000000840|0|final theodolites alongside of the carefu|61.6141|-149.457|61.6141|-149.457|
-1316|16|F|163746.47|1993-12-03|1-URGENT|Clerk#000000857|0|ully bold theodolites? pending, bold pin|61.5969|-149.367|61.5969|-149.367|
-1317|100|P|139714.71|1995-05-19|2-HIGH|Clerk#000000373|0|sts. furiously special deposits lose fur|61.58|-149.4|61.58|-149.4|
-1318|128|O|81663.65|1998-06-27|3-MEDIUM|Clerk#000000581|0|s hang bold requests. pending, re|61.5848|-149.445|61.5848|-149.445|
-1319|32|O|31103.83|1996-09-27|2-HIGH|Clerk#000000257|0|y across the ruthlessly ironic accounts. unusu|61.5811|-149.444|61.5811|-149.444|
-1344|17|F|43809.37|1992-04-16|5-LOW|Clerk#000000178|0|omise close, silent requests. pending theodolites boost pending |61.5733|-149.389|61.5733|-149.389|
-1345|95|F|111207.93|1992-10-28|5-LOW|Clerk#000000447|0| regular tithes. quickly fluffy de|61.6141|-149.457|61.6141|-149.457|
-1346|76|F|171975.62|1992-06-18|2-HIGH|Clerk#000000374|0|ges sleep quickly-- even pint|61.5952|-149.436|61.5952|-149.436|
-1347|41|O|173444.60|1997-06-20|5-LOW|Clerk#000000977|0|he furiously even foxes use carefully express req|61.5421|-149.419|61.5421|-149.419|
-1348|19|O|94135.77|1998-04-18|5-LOW|Clerk#000000206|0|tly. quickly even deposi|61.5783|-149.362|61.5783|-149.362|
-1349|64|O|46376.09|1997-10-26|1-URGENT|Clerk#000000543|0|yly! blithely special theodolites cajole. unusual, reg|61.7321|-150.12|61.7321|-150.12|
-1350|52|F|49305.98|1993-08-24|1-URGENT|Clerk#000000635|0|iously about the blithely special a|61.5691|-149.328|61.5691|-149.328|
-1351|106|O|24637.96|1998-04-20|1-URGENT|Clerk#000000012|0| cajole. regular, special re|61.6141|-149.457|61.6141|-149.457|
-1376|47|O|23984.88|1997-05-04|4-NOT SPECIFIED|Clerk#000000730|0|der furiously final, final frets. carefull|61.5819|-149.3|61.5819|-149.3|
-1377|20|O|108334.30|1998-04-24|4-NOT SPECIFIED|Clerk#000000625|0|lly across the blithely express accounts. ironic excuses promise carefully de|61.6431|-149.289|61.6431|-149.289|
-1378|20|O|118495.12|1996-03-09|4-NOT SPECIFIED|Clerk#000000705|0| furiously even tithes cajole slyly among the quick|61.6431|-149.292|61.6431|-149.292|
-1379|65|O|84627.76|1998-05-25|5-LOW|Clerk#000000861|0|y deposits are caref|61.6228|-149.313|61.6228|-149.313|
-1380|137|O|94969.41|1996-07-07|3-MEDIUM|Clerk#000000969|0|inal deposits wake slyly daringly even requests. bold, even foxe|61.2125|-149.894|61.2125|-149.894|
-1381|127|O|58212.22|1998-05-25|3-MEDIUM|Clerk#000000107|0|even requests breach after the bold, ironic instructions. slyly even|61.1879|-149.886|61.1879|-149.886|
-1382|133|F|173522.71|1993-08-17|5-LOW|Clerk#000000241|0|fully final packages sl|61.1594|-149.835|61.1594|-149.835|
-1383|121|F|34797.72|1993-04-27|2-HIGH|Clerk#000000785|0|ts. express requests sleep blithel|61.2123|-149.854|61.2123|-149.854|
-1408|55|O|183965.61|1997-12-26|4-NOT SPECIFIED|Clerk#000000942|0|t the quickly final asymptotes. unusual|61.1951|-149.945|61.1951|-149.945|
-1409|143|F|72440.52|1992-12-31|4-NOT SPECIFIED|Clerk#000000065|0|ructions. furiously unusual excuses are regular, unusual theodolites. fin|61.2138|-149.856|61.2138|-149.856|
-1410|113|O|114879.19|1997-04-12|5-LOW|Clerk#000000123|0|iously along the bravely regular dolphins. pinto beans cajole furiously sp|61.1255|-149.864|61.1255|-149.864|
-1411|95|F|164462.61|1994-12-21|2-HIGH|Clerk#000000566|0|s. furiously special excuses across the pending pinto beans haggle sp|61.2066|-149.808|61.2066|-149.808|
-1412|53|F|78676.54|1993-03-13|4-NOT SPECIFIED|Clerk#000000083|0|uffily daring theodolit|61.2138|-149.896|61.2138|-149.896|
-1413|91|O|75733.58|1997-06-14|3-MEDIUM|Clerk#000000342|0|, ironic instructions. carefully even packages dazzle|61.2161|-149.876|61.2161|-149.876|
-1414|77|O|38057.81|1995-08-16|1-URGENT|Clerk#000000883|0|ccounts. ironic foxes haggle car|61.1594|-149.888|61.1594|-149.888|
-1415|79|F|24654.79|1994-05-29|4-NOT SPECIFIED|Clerk#000000601|0|rays. blithely final ideas affix quickl|61.1806|-149.775|61.1806|-149.775|
-1440|98|O|50201.16|1995-08-10|5-LOW|Clerk#000000956|0| pending requests. closely s|61.1101|-149.857|61.1101|-149.857|
-1441|122|O|156477.94|1997-03-06|4-NOT SPECIFIED|Clerk#000000156|0|ter the excuses. ironic dependencies m|61.1541|-149.958|61.1541|-149.958|
-1442|112|F|7108.12|1994-07-05|4-NOT SPECIFIED|Clerk#000000935|0|nal pinto beans. slyly ironic ideas cajol|61.1268|-149.947|61.1268|-149.947|
-1443|44|O|44672.03|1996-12-16|5-LOW|Clerk#000000185|0|x blithely against the carefully final somas. even asymptotes are. quickly spe|61.0931|-149.785|61.0931|-149.785|
-1444|134|F|207907.60|1994-12-06|3-MEDIUM|Clerk#000000783|0|ove the bold accounts cajole fluffily about|61.1901|-149.892|61.1901|-149.892|
-1445|115|F|154653.32|1995-01-10|3-MEDIUM|Clerk#000000211|0|even packages wake fluffily |61.2183|-149.889|61.2183|-149.889|
-1446|41|O|27663.16|1998-02-16|5-LOW|Clerk#000000274|0|lly regular notornis above the requests sleep final accounts! |61.2164|-149.882|61.2164|-149.882|
-1447|91|F|108171.38|1992-10-15|2-HIGH|Clerk#000000880|0|inly against the blithely pending excuses. regular, pe|61.2161|-149.876|61.2161|-149.876|
-1472|149|O|65331.05|1996-10-06|5-LOW|Clerk#000000303|0|y special dolphins around the final dependencies wake quick|61.219|-149.792|61.219|-149.792|
-1473|94|O|80624.38|1997-03-17|3-MEDIUM|Clerk#000000960|0|furiously close accoun|61.2188|-149.892|61.2188|-149.892|
-1474|70|F|51697.18|1995-01-09|1-URGENT|Clerk#000000438|0|detect quickly above the carefully even |61.2143|-149.837|61.2143|-149.837|
-1475|5|O|185496.66|1997-11-12|2-HIGH|Clerk#000000972|0|cally final packages boost. blithely ironic packa|61.1608|-149.835|61.1608|-149.835|
-1476|145|O|18795.62|1996-06-27|2-HIGH|Clerk#000000673|0|ding accounts hinder alongside of the quickly pending requests. fluf|61.1886|-149.944|61.1886|-149.944|
-1477|76|O|231831.35|1997-08-24|5-LOW|Clerk#000000612|0|ly bold foxes. final ideas would cajo|61.1201|-149.89|61.1201|-149.89|
-1478|50|O|20791.50|1997-08-03|2-HIGH|Clerk#000000827|0|lessly. carefully express|61.1201|-149.89|61.1201|-149.89|
-1479|16|O|31471.04|1995-12-16|4-NOT SPECIFIED|Clerk#000000697|0|he furiously even foxes. thinly bold deposits|61.1585|-149.872|61.1585|-149.872|
-1504|2|F|89399.40|1992-08-28|3-MEDIUM|Clerk#000000381|0|, brave deposits. bold de|61.195|-149.892|61.195|-149.892|
-1505|37|F|55892.35|1992-08-21|2-HIGH|Clerk#000000544|0|s. slyly ironic packages cajole. carefully regular packages haggle |61.0895|-149.694|61.0895|-149.694|
-1506|148|F|195844.84|1992-09-21|3-MEDIUM|Clerk#000000620|0| dependencies. accounts affix blithely slowly unusual deposits. slyly regular |61.2201|-149.831|61.2201|-149.831|
-1507|121|F|96166.92|1993-10-14|3-MEDIUM|Clerk#000000305|0|stealthy, ironic de|61.1663|-149.867|61.1663|-149.867|
-1508|103|O|151282.65|1998-04-10|5-LOW|Clerk#000000117|0| after the furiously regular pinto beans hang slyly quickly ironi|61.2138|-149.906|61.2138|-149.906|
-1509|64|F|180455.98|1993-07-08|5-LOW|Clerk#000000770|0|the regular ideas. regul|61.2193|-149.902|61.2193|-149.902|
-1510|53|O|154590.05|1996-09-17|5-LOW|Clerk#000000128|0|ld carefully. furiously final asymptotes haggle furiously|61.1201|-149.89|61.1201|-149.89|
-1511|79|O|59651.38|1996-12-22|4-NOT SPECIFIED|Clerk#000000386|0|ts above the depend|61.1601|-149.984|61.1601|-149.984|
-1536|94|O|5184.26|1997-01-26|3-MEDIUM|Clerk#000000117|0|ges are! furiously final deposits cajole iron|61.1101|-149.857|61.1101|-149.857|
-1537|109|F|108317.51|1992-02-15|4-NOT SPECIFIED|Clerk#000000862|0|g to the even deposits. ironic, final packages |61.1101|-149.857|61.1101|-149.857|
-1538|29|O|179554.41|1995-06-18|4-NOT SPECIFIED|Clerk#000000258|0| instructions. regular theod|61.1101|-149.857|61.1101|-149.857|
-1539|112|F|39612.63|1995-03-10|5-LOW|Clerk#000000840|0|nstructions boost pa|61.1101|-149.857|61.1101|-149.857|
-1540|16|F|128014.15|1992-08-05|2-HIGH|Clerk#000000927|0|r ideas hinder blithe|61.1101|-149.857|61.1101|-149.857|
-1541|94|P|47286.32|1995-05-18|1-URGENT|Clerk#000000906|0|y. slyly ironic warhorses around the furiously regul|61.1101|-149.857|61.1101|-149.857|
-1542|143|F|132972.24|1993-09-15|3-MEDIUM|Clerk#000000435|0|t the furiously close deposits do was f|61.1101|-149.857|61.1101|-149.857|
-1543|52|O|139047.22|1997-02-20|1-URGENT|Clerk#000000398|0|unts. furiously pend|61.1101|-149.857|61.1101|-149.857|
-1568|17|O|76119.72|1997-01-30|4-NOT SPECIFIED|Clerk#000000554|0|d notornis. carefully |61.1101|-149.857|61.1101|-149.857|
-1569|104|O|87803.55|1998-04-02|5-LOW|Clerk#000000786|0|orbits. fluffily even decoys serve blithely. furiously furious realms nag acro|61.1101|-149.857|61.1101|-149.857|
-1570|124|O|35589.57|1998-03-16|1-URGENT|Clerk#000000745|0|pinto beans haggle furiousl|61.1101|-149.857|61.1101|-149.857|
-1571|103|F|151404.78|1992-12-05|2-HIGH|Clerk#000000565|0|ously furiously bold warthogs. slyly ironic instructions are quickly a|61.1101|-149.857|61.1101|-149.857|
-1572|11|O|47232.79|1996-02-24|2-HIGH|Clerk#000000994|0|fluffily ironic accounts haggle blithely final platelets! slyly regular foxes|61.1101|-149.857|61.1101|-149.857|
-1573|148|F|86918.57|1992-12-28|2-HIGH|Clerk#000000940|0|ess, ironic deposits use along the carefu|61.1101|-149.857|61.1101|-149.857|
-1574|134|O|179923.54|1996-12-12|3-MEDIUM|Clerk#000000809|0| ideas hinder after the carefully unusual |61.1101|-149.857|61.1101|-149.857|
-1575|145|O|197031.52|1995-09-13|3-MEDIUM|Clerk#000000497|0|. furiously regular dep|61.1101|-149.857|61.1101|-149.857|
-1600|94|F|130515.61|1993-03-03|3-MEDIUM|Clerk#000000627|0|tions cajole quietly above the regular, silent requests. slyly fin|61.1101|-149.857|61.1101|-149.857|
-1601|53|F|73962.95|1994-08-27|5-LOW|Clerk#000000469|0|ent deposits are ca|61.1101|-149.857|61.1101|-149.857|
-1602|1|F|4225.26|1993-08-05|5-LOW|Clerk#000000660|0|deposits. busily silent instructions haggle furiously. fin|61.1101|-149.857|61.1101|-149.857|
-1603|2|F|29305.47|1993-07-31|4-NOT SPECIFIED|Clerk#000000869|0|s. slyly silent deposits boo|61.1101|-149.857|61.1101|-149.857|
-1604|113|F|107139.29|1993-07-17|5-LOW|Clerk#000000512|0|lithely silent waters. blithely unusual packages alongside |61.1101|-149.857|61.1101|-149.857|
-1605|58|O|130687.64|1998-04-24|4-NOT SPECIFIED|Clerk#000000616|0|sleep furiously? ruthless, even pinto beans |61.1101|-149.857|61.1101|-149.857|
-1606|53|O|115877.40|1997-04-17|4-NOT SPECIFIED|Clerk#000000550|0|r requests. quickly even platelets breach before the ironically|61.1101|-149.857|61.1101|-149.857|
-1607|149|O|166335.03|1995-12-16|2-HIGH|Clerk#000000498|0| bold, pending foxes haggle. slyly silent |61.1101|-149.857|61.1101|-149.857|
-1632|67|O|183286.33|1997-01-08|3-MEDIUM|Clerk#000000351|0|onic requests are accounts. bold a|61.1101|-149.857|61.1101|-149.857|
-1633|16|O|52359.51|1995-10-14|2-HIGH|Clerk#000000666|0|y silent accounts sl|61.1101|-149.857|61.1101|-149.857|
-1634|70|O|145898.47|1996-09-10|1-URGENT|Clerk#000000360|0|arefully blithely ironic requests. slyly unusual instructions alongside|61.1101|-149.857|61.1101|-149.857|
-1635|4|O|70232.26|1997-02-13|3-MEDIUM|Clerk#000000958|0|s. slyly ironic requests affix slyly |61.1101|-149.857|61.1101|-149.857|
-1636|79|O|172021.87|1997-06-17|3-MEDIUM|Clerk#000000457|0|ding requests. slyly ironic courts wake quickl|61.1101|-149.857|61.1101|-149.857|
-1637|73|F|180912.15|1995-02-08|4-NOT SPECIFIED|Clerk#000000189|0| final accounts. blithely silent ideas cajole bravely. carefully express |61.1101|-149.857|61.1101|-149.857|
-1638|139|O|172436.30|1997-08-13|2-HIGH|Clerk#000000643|0|he fluffily regular asymp|61.1101|-149.857|61.1101|-149.857|
-1639|5|O|104166.56|1995-08-20|4-NOT SPECIFIED|Clerk#000000939|0|haggle furiously. final requests detect furious|61.1101|-149.857|61.1101|-149.857|
-1664|64|O|178060.22|1996-03-03|1-URGENT|Clerk#000000090|0|y quickly even asymptotes. furiously regular packages haggle quickly fin|61.1101|-149.857|61.1101|-149.857|
-1665|76|F|4819.91|1994-05-08|2-HIGH|Clerk#000000920|0|ly regular packages are fluffily even ideas. fluffily final|61.1101|-149.857|61.1101|-149.857|
-1666|95|O|128367.97|1995-10-18|1-URGENT|Clerk#000000849|0|ffily pending dependencies wake fluffily. pending, final accounts |61.1101|-149.857|61.1101|-149.857|
-1667|5|O|125030.37|1997-10-10|2-HIGH|Clerk#000000103|0|e accounts. slyly express accounts must are a|64.8459|-147.759|64.8459|-147.759|
-1668|142|O|137576.19|1997-07-12|4-NOT SPECIFIED|Clerk#000000148|0|eodolites. carefully dogged dolphins haggle q|64.8426|-147.725|64.8426|-147.725|
-1669|2|O|24362.39|1997-06-09|3-MEDIUM|Clerk#000000663|0|er ironic requests detect furiously blithely sp|64.9401|-147.402|64.9401|-147.402|
-1670|25|O|89999.72|1997-05-24|2-HIGH|Clerk#000000320|0|unusual dependencies. furiously special platelets main|64.9401|-147.402|64.9401|-147.402|
-1671|35|O|104391.11|1996-07-27|4-NOT SPECIFIED|Clerk#000000275|0|ly. slyly pending requests was above the |64.8331|-147.647|64.8331|-147.647|
-1696|4|O|102665.03|1998-01-08|4-NOT SPECIFIED|Clerk#000000041|0|bravely bold accounts above the quickly bold|64.8371|-147.716|64.8371|-147.716|
-1697|76|O|122621.31|1996-10-07|1-URGENT|Clerk#000000815|0|o x-ray blithely. pl|64.8574|-147.759|64.8574|-147.759|
-1698|40|O|141118.87|1997-04-23|2-HIGH|Clerk#000000432|0|slyly. carefully express deposit|64.836|-147.727|64.836|-147.727|
-1699|85|F|66408.29|1993-12-30|1-URGENT|Clerk#000000125|0|jole blithely. furiously un|64.8132|-147.76|64.8132|-147.76|
-1700|65|O|89143.36|1996-06-15|3-MEDIUM|Clerk#000000328|0|ely final dolphins wake sometimes above the quietly regular deposits. fur|64.8451|-147.96|64.8451|-147.96|
-1701|130|F|72835.95|1992-05-19|2-HIGH|Clerk#000000395|0|furiously. regular, close theodoli|64.8891|-147.851|64.8891|-147.851|
-1702|67|P|194119.31|1995-05-07|2-HIGH|Clerk#000000300|0|around the carefully final deposits cajole carefully according to the b|64.8151|-147.707|64.8151|-147.707|
-1703|134|F|121220.59|1993-01-28|3-MEDIUM|Clerk#000000463|0| pinto beans poach. bold courts boost. regular, express deposits at|64.8363|-147.803|64.8363|-147.803|
-1728|64|O|131604.34|1996-05-22|2-HIGH|Clerk#000000711|0|beans. slyly regular instructions sleep! slyly final packages|64.8298|-147.611|64.8298|-147.611|
-1729|133|F|12137.76|1992-05-19|2-HIGH|Clerk#000000158|0|pending foxes wake. accounts|64.8989|-147.701|64.8989|-147.701|
-1730|124|O|150886.49|1998-07-24|5-LOW|Clerk#000000794|0| fluffily pending deposits serve. furiously even requests wake furiou|64.8371|-147.716|64.8371|-147.716|
-1731|128|O|190490.78|1996-01-06|1-URGENT|Clerk#000000268|0|lithely regular, final instructions. ironic, express packages are above|64.8147|-147.706|64.8147|-147.706|
-1732|146|F|179854.51|1993-11-29|5-LOW|Clerk#000000903|0|inal requests integrate dolph|64.8451|-147.812|64.8451|-147.812|
-1733|148|O|165489.52|1996-05-12|2-HIGH|Clerk#000000789|0|e carefully according to the accounts. furiously pending instructions sleep|64.8386|-147.788|64.8386|-147.788|
-1734|7|F|44002.53|1994-06-11|2-HIGH|Clerk#000000722|0| final ideas haggle. blithely quick foxes sleep busily bold ideas. i|64.8372|-147.768|64.8372|-147.768|
-1735|22|F|98541.95|1992-12-27|1-URGENT|Clerk#000000458|0|ully idle requests wake qu|64.8151|-147.707|64.8151|-147.707|
-1760|115|O|82151.12|1996-05-17|5-LOW|Clerk#000000917|0| deposits. busily regular deposits wake blithely along the furiously even re|64.843|-147.722|64.843|-147.722|
-1761|106|F|211925.95|1993-12-24|2-HIGH|Clerk#000000817|0|efully slyly bold frets. packages boost b|64.8426|-147.725|64.8426|-147.725|
-1762|77|F|202227.17|1994-08-20|4-NOT SPECIFIED|Clerk#000000653|0|ly ironic packages. furi|64.8615|-147.723|64.8615|-147.723|
-1763|121|O|140685.01|1996-10-29|2-HIGH|Clerk#000000321|0|es. bold dependencies haggle furiously along |64.8694|-147.067|64.8694|-147.067|
-1764|29|F|47384.71|1992-03-25|1-URGENT|Clerk#000000182|0|. slyly final packages integrate carefully acro|64.8404|-147.724|64.8404|-147.724|
-1765|73|O|36551.43|1995-12-03|4-NOT SPECIFIED|Clerk#000000490|0| regular excuses wake slyly|64.9686|-147.577|64.9686|-147.577|
-1766|139|O|41032.81|1996-10-12|2-HIGH|Clerk#000000983|0|unusual deposits affix quickly beyond the carefully s|64.8497|-147.732|64.8497|-147.732|
-1767|25|P|136582.60|1995-03-14|2-HIGH|Clerk#000000327|0|eposits use carefully carefully regular platelets. quickly regular packages al|64.8861|-147.587|64.8861|-147.587|
-1792|49|F|107919.86|1993-11-09|5-LOW|Clerk#000000102|0|ructions haggle along the pending packages. carefully speci|64.8508|-147.703|64.8508|-147.703|
-1793|19|F|82504.56|1992-07-12|4-NOT SPECIFIED|Clerk#000000291|0|regular packages cajole. blithely special packages according to the final d|64.841|-147.72|64.841|-147.72|
-1794|140|O|179462.21|1997-09-28|1-URGENT|Clerk#000000686|0|ally silent pinto beans. regular package|64.8375|-147.721|64.8375|-147.721|
-1795|94|F|146849.33|1994-03-19|2-HIGH|Clerk#000000815|0| quickly final packages! blithely dogged accounts c|64.849|-147.813|64.849|-147.813|
-1796|47|F|33755.47|1992-11-21|2-HIGH|Clerk#000000245|0|eans use furiously around th|55.3801|-131.682|55.3801|-131.682|
-1797|125|O|51494.47|1996-05-07|3-MEDIUM|Clerk#000000508|0|quiet platelets haggle since the quickly ironic instructi|55.3801|-131.682|55.3801|-131.682|
-1798|52|O|46393.97|1997-07-28|1-URGENT|Clerk#000000741|0|al foxes are blithe|55.3603|-131.702|55.3603|-131.702|
-1799|61|F|46815.93|1994-03-07|4-NOT SPECIFIED|Clerk#000000339|0|ns sleep furiously final waters. blithely regular instructions h|55.7511|-132.865|55.7511|-132.865|
-1824|49|F|81351.53|1994-05-05|1-URGENT|Clerk#000000972|0|e blithely fluffily|55.7511|-132.865|55.7511|-132.865|
-1825|148|F|150582.77|1993-12-05|3-MEDIUM|Clerk#000000345|0|ironic, final accou|60.3311|-151.284|60.3311|-151.284|
-1826|82|F|124719.97|1992-04-16|4-NOT SPECIFIED|Clerk#000000718|0|the even asymptotes dazzle fluffily slyly regular asymptotes. final, unu|60.3311|-151.284|60.3311|-151.284|
-1827|106|O|210113.88|1996-06-22|4-NOT SPECIFIED|Clerk#000000369|0|luffily even requests haggle sly|60.3311|-151.284|60.3311|-151.284|
-1828|32|F|137369.50|1994-04-18|3-MEDIUM|Clerk#000000840|0|y quickly bold packag|60.4341|-151.283|60.4341|-151.283|
-1829|112|F|127532.20|1994-05-08|2-HIGH|Clerk#000000537|0| accounts wake above the furiously unusual requests. pending package|60.3311|-151.284|60.3311|-151.284|
-1830|133|F|85122.24|1995-02-23|1-URGENT|Clerk#000000045|0|according to the even,|60.3311|-151.284|60.3311|-151.284|
-1831|71|F|58032.77|1993-12-02|1-URGENT|Clerk#000000854|0| accounts. carefully even accounts boost furiously. regular ideas engage. |60.3311|-151.284|60.3311|-151.284|
-1856|106|F|189361.42|1992-03-20|4-NOT SPECIFIED|Clerk#000000952|0|. special pinto beans run acr|60.3311|-151.284|60.3311|-151.284|
-1857|133|F|102793.59|1993-01-13|2-HIGH|Clerk#000000083|0|hely final ideas slee|60.3311|-151.284|60.3311|-151.284|
-1858|143|O|30457.91|1997-12-13|1-URGENT|Clerk#000000389|0|thely. slyly final deposits sleep|60.4311|-151.286|60.4311|-151.286|
-1859|61|O|105094.09|1997-04-11|4-NOT SPECIFIED|Clerk#000000949|0| the foxes. bravely special excuses nag carefully special r|60.3311|-151.284|60.3311|-151.284|
-1860|10|O|9103.40|1996-04-04|3-MEDIUM|Clerk#000000556|0|osits. quickly bold deposits according to |60.3311|-151.284|60.3311|-151.284|
-1861|70|F|95063.41|1994-01-03|3-MEDIUM|Clerk#000000847|0|r the fluffily close sauternes. furio|60.3311|-151.284|60.3311|-151.284|
-1862|34|O|97981.06|1998-02-24|5-LOW|Clerk#000000348|0|ts snooze ironically abou|60.3311|-151.284|60.3311|-151.284|
-1863|74|F|96359.65|1993-09-23|4-NOT SPECIFIED|Clerk#000000658|0|old sentiments. careful, |60.3191|-151.296|60.3191|-151.296|
-1888|121|F|224724.11|1993-10-31|4-NOT SPECIFIED|Clerk#000000659|0|olites. pinto beans cajole. regular deposits affix. slyly regular|60.3311|-151.284|60.3311|-151.284|
-1889|25|O|96431.77|1997-03-16|1-URGENT|Clerk#000000854|0|p around the regular notornis. unusual deposits|60.3311|-151.284|60.3311|-151.284|
-1890|10|O|202364.58|1996-12-18|4-NOT SPECIFIED|Clerk#000000627|0|romise final, regular deposits. regular fox|60.3311|-151.284|60.3311|-151.284|
-1891|61|F|76848.96|1994-12-15|5-LOW|Clerk#000000495|0|unusual foxes sleep regular deposits. requests wake special pac|60.5563|-151.241|60.5563|-151.241|
-1892|25|F|133273.64|1994-03-26|5-LOW|Clerk#000000733|0|sts. slyly regular dependencies use slyly. ironic, spec|60.6331|-151.163|60.6331|-151.163|
-1893|125|O|116792.13|1997-10-30|2-HIGH|Clerk#000000111|0|olites. silent, special deposits eat slyly quickly express packages; hockey p|60.6331|-151.163|60.6331|-151.163|
-1894|76|F|44387.23|1992-03-30|1-URGENT|Clerk#000000626|0|e furiously. furiously even accounts are slyly final accounts. closely speci|60.6331|-151.163|60.6331|-151.163|
-1895|7|F|44429.81|1994-05-30|3-MEDIUM|Clerk#000000878|0|ress accounts. bold accounts cajole. slyly final pinto beans poach regul|60.6331|-151.163|60.6331|-151.163|
-1920|110|O|119605.91|1998-06-24|5-LOW|Clerk#000000018|0|hely; furiously regular excuses|60.5551|-151.245|60.5551|-151.245|
-1921|88|F|57584.12|1994-01-18|3-MEDIUM|Clerk#000000293|0|counts. slyly quiet requests along the ruthlessly regular accounts are |60.6331|-151.163|60.6331|-151.163|
-1922|56|O|11575.77|1996-07-13|3-MEDIUM|Clerk#000000984|0|side of the blithely final re|60.5506|-151.141|60.5506|-151.141|
-1923|136|O|171128.10|1997-07-07|1-URGENT|Clerk#000000471|0| express dolphins. |60.5681|-151.281|60.5681|-151.281|
-1924|76|O|169756.19|1996-09-07|4-NOT SPECIFIED|Clerk#000000823|0| of the ironic accounts. instructions near the final instr|60.5465|-151.147|60.5465|-151.147|
-1925|17|F|146382.71|1992-03-05|1-URGENT|Clerk#000000986|0|e slyly regular deposits. furiously |60.6331|-151.163|60.6331|-151.163|
-1926|94|O|100035.03|1996-01-31|2-HIGH|Clerk#000000568|0|cajole. even warhorses sleep carefully. |60.5578|-151.116|60.5578|-151.116|
-1927|140|O|23327.88|1995-09-30|3-MEDIUM|Clerk#000000616|0|riously special packages. permanent pearls wake furiously. even packages alo|61.6182|-149.385|61.6182|-149.385|
-1952|67|F|12896.25|1994-03-16|2-HIGH|Clerk#000000254|0| silent accounts boost |61.6141|-149.457|61.6141|-149.457|
-1953|149|F|57213.18|1993-11-30|3-MEDIUM|Clerk#000000891|0| fluffily along the quickly even packages. |61.5765|-149.407|61.5765|-149.407|
-1954|56|O|158853.63|1997-05-31|4-NOT SPECIFIED|Clerk#000000104|0| unusual excuses cajole according to the blithely regular theodolites.|61.6091|-149.77|61.6091|-149.77|
-1955|13|F|103085.13|1992-04-20|1-URGENT|Clerk#000000792|0|ly special ideas. sometimes final |61.5821|-149.438|61.5821|-149.438|
-1956|127|F|88704.26|1992-09-20|4-NOT SPECIFIED|Clerk#000000600|0|ironic ideas are silent ideas. furiously final deposits sleep slyly carefu|61.6183|-149.373|61.6183|-149.373|
-1957|31|O|77482.87|1998-07-21|2-HIGH|Clerk#000000639|0|nding excuses about the |61.6131|-149.403|61.6131|-149.403|
-1958|53|O|176294.34|1995-09-22|5-LOW|Clerk#000000343|0| haggle blithely. flu|61.6352|-149.265|61.6352|-149.265|
-1959|43|O|62277.18|1997-01-13|4-NOT SPECIFIED|Clerk#000000631|0| cajole about the blithely express requests. even excuses mold bl|61.5751|-149.645|61.5751|-149.645|
-1984|52|O|79230.47|1998-04-01|1-URGENT|Clerk#000000416|0| slyly special instructions. unusual foxes use packages. carefully regular req|61.6168|-149.374|61.6168|-149.374|
-1985|7|F|171522.54|1994-09-02|4-NOT SPECIFIED|Clerk#000000741|0|slyly slyly even pains. slyly reg|61.5939|-149.43|61.5939|-149.43|
-1986|149|F|34269.96|1994-05-05|2-HIGH|Clerk#000000609|0|across the theodolites. quick|61.5792|-149.495|61.5792|-149.495|
-1987|100|F|6406.29|1994-04-30|2-HIGH|Clerk#000000652|0|gular platelets alongside |61.6141|-149.457|61.6141|-149.457|
-1988|109|O|117132.72|1995-10-06|4-NOT SPECIFIED|Clerk#000000011|0|ly ironic dolphins serve quickly busy accounts. bu|61.5829|-149.448|61.5829|-149.448|
-1989|118|F|39263.28|1994-03-16|4-NOT SPECIFIED|Clerk#000000747|0|ely bold pinto beans ha|61.5938|-149.387|61.5938|-149.387|
-1990|119|F|48781.39|1994-12-16|2-HIGH|Clerk#000000114|0|e bold patterns. always regul|61.5849|-149.38|61.5849|-149.38|
-1991|19|F|139854.41|1992-09-07|4-NOT SPECIFIED|Clerk#000000854|0|ing accounts can haggle at the carefully final Tiresias-- pending, regular|61.5729|-149.389|61.5729|-149.389|
-2016|8|O|24347.36|1996-08-16|3-MEDIUM|Clerk#000000641|0|the carefully ironic foxes. requests nag bold, r|61.5823|-149.462|61.5823|-149.462|
-2017|101|O|70529.27|1998-05-13|3-MEDIUM|Clerk#000000427|0|nusual requests. blit|61.57|-149.331|61.57|-149.331|
-2018|19|P|25007.95|1995-04-05|4-NOT SPECIFIED|Clerk#000000920|0|gular accounts wake fur|61.5821|-149.438|61.5821|-149.438|
-2019|136|F|43789.14|1992-10-23|1-URGENT|Clerk#000000565|0| furiously bold packages. fluffily fi|61.6141|-149.457|61.6141|-149.457|
-2020|73|F|136162.13|1993-06-21|3-MEDIUM|Clerk#000000192|0|es. furiously regular packages above the furiously special theodolites are a|61.6115|-149.331|61.6115|-149.331|
-2021|70|O|27016.74|1995-07-15|1-URGENT|Clerk#000000155|0|ong the furiously regular requests. unusual deposits wake fluffily inside|61.6091|-149.77|61.6091|-149.77|
-2022|62|F|206742.11|1992-03-15|1-URGENT|Clerk#000000268|0| dependencies sleep fluffily even, ironic deposits. express, silen|61.6141|-149.457|61.6141|-149.457|
-2023|118|F|144123.37|1992-05-06|5-LOW|Clerk#000000137|0|ular courts engage according to the|61.5826|-149.427|61.5826|-149.427|
-2048|17|F|33401.77|1993-11-15|1-URGENT|Clerk#000000934|0|s cajole after the blithely final accounts. f|61.5976|-149.366|61.5976|-149.366|
-2049|31|O|153048.74|1995-12-07|2-HIGH|Clerk#000000859|0|ly regular requests thrash blithely about the fluffily even theodolites. r|61.5976|-149.366|61.5976|-149.366|
-2050|28|F|208517.98|1994-06-02|4-NOT SPECIFIED|Clerk#000000821|0|d accounts against the furiously regular packages use bli|61.5531|-149.651|61.5531|-149.651|
-2051|40|O|87988.34|1996-03-18|4-NOT SPECIFIED|Clerk#000000333|0|ctions sleep blithely. blithely regu|61.5531|-149.651|61.5531|-149.651|
-2052|91|F|141822.19|1992-04-13|2-HIGH|Clerk#000000767|0| requests sleep around the even, even courts. ironic theodolites affix furious|61.5883|-149.456|61.5883|-149.456|
-2053|142|F|125125.57|1995-02-07|1-URGENT|Clerk#000000717|0|ar requests: blithely sly accounts boost carefully across t|61.6249|-149.435|61.6249|-149.435|
-2054|41|F|144335.16|1992-06-08|4-NOT SPECIFIED|Clerk#000000103|0|l requests affix carefully about the furiously special|61.6141|-149.457|61.6141|-149.457|
-2055|97|F|57092.26|1993-09-04|1-URGENT|Clerk#000000067|0|. warhorses affix slyly blithely express instructions? fur|61.5709|-149.452|61.5709|-149.452|
-2080|95|F|45767.69|1993-06-18|5-LOW|Clerk#000000190|0|ironic, pending theodolites are carefully about the quickly regular theodolite|61.6651|-149.465|61.6651|-149.465|
-2081|121|O|145654.97|1997-07-05|2-HIGH|Clerk#000000136|0|ong the regular theo|61.5841|-149.441|61.5841|-149.441|
-2082|49|F|46753.63|1995-01-10|2-HIGH|Clerk#000000354|0|cial accounts. ironic, express dolphins nod slyly sometimes final reques|61.1571|-149.883|61.1571|-149.883|
-2083|101|F|31795.52|1993-07-14|3-MEDIUM|Clerk#000000361|0|al patterns. bold, final foxes nag bravely about the furiously express|61.2198|-149.733|61.2198|-149.733|
-2084|80|F|190652.53|1993-03-17|2-HIGH|Clerk#000000048|0|zle furiously final, careful packages. slyly ironic ideas amo|61.1863|-149.976|61.1863|-149.976|
-2085|49|F|45311.07|1993-11-21|3-MEDIUM|Clerk#000000818|0|ress, express ideas haggle|61.2161|-149.876|61.2161|-149.876|
-2086|142|F|188985.18|1994-10-19|1-URGENT|Clerk#000000046|0| permanently regular|61.2031|-149.749|61.2031|-149.749|
-2087|50|O|53581.41|1998-01-31|2-HIGH|Clerk#000000626|0|e always regular packages nod against the furiously spec|61.1644|-149.897|61.1644|-149.897|
-2112|64|O|17986.15|1997-02-05|2-HIGH|Clerk#000000351|0|against the slyly even id|61.1834|-149.866|61.1834|-149.866|
-2113|32|O|65678.21|1997-11-08|2-HIGH|Clerk#000000527|0|slyly regular instruct|61.1731|-149.889|61.1731|-149.889|
-2114|79|F|106446.02|1995-01-16|5-LOW|Clerk#000000751|0|r, unusual accounts haggle across the busy platelets. carefully |61.1089|-149.854|61.1089|-149.854|
-2115|106|O|134814.65|1998-05-23|4-NOT SPECIFIED|Clerk#000000101|0|odolites boost. carefully regular excuses cajole. quickly ironic pinto be|61.1951|-149.916|61.1951|-149.916|
-2116|23|F|60887.90|1994-08-26|1-URGENT|Clerk#000000197|0|efully after the asymptotes. furiously sp|61.2157|-149.821|61.2157|-149.821|
-2117|22|O|145713.03|1997-04-26|2-HIGH|Clerk#000000887|0|ely even dependencies. regular foxes use blithely.|61.1372|-149.954|61.1372|-149.954|
-2118|134|O|38974.67|1996-10-09|1-URGENT|Clerk#000000196|0|ial requests wake carefully special packages. f|61.1955|-149.737|61.1955|-149.737|
-2119|64|O|34632.57|1996-08-20|2-HIGH|Clerk#000000434|0|uickly pending escapades. fluffily ir|61.1444|-149.867|61.1444|-149.867|
-2144|136|F|119917.28|1994-03-29|3-MEDIUM|Clerk#000000546|0|t. carefully quick requests across the deposits wake regu|61.2178|-149.882|61.2178|-149.882|
-2145|134|F|18885.35|1992-10-03|1-URGENT|Clerk#000000886|0|sts would snooze blithely alongside of th|61.1824|-149.849|61.1824|-149.849|
-2146|118|F|179686.07|1992-09-14|4-NOT SPECIFIED|Clerk#000000476|0|ven packages. dependencies wake slyl|61.2161|-149.876|61.2161|-149.876|
-2147|100|F|91513.79|1992-09-06|4-NOT SPECIFIED|Clerk#000000424|0| haggle carefully furiously final foxes. pending escapades thrash. bold theod|61.2022|-149.84|61.2022|-149.84|
-2148|130|F|19612.03|1995-04-19|4-NOT SPECIFIED|Clerk#000000517|0|ross the furiously unusual theodolites. always expre|61.2099|-149.762|61.2099|-149.762|
-2149|101|F|105145.40|1993-03-13|5-LOW|Clerk#000000555|0|nusual accounts nag furiously special reques|61.1951|-149.84|61.1951|-149.84|
-2150|82|F|166961.06|1994-06-03|3-MEDIUM|Clerk#000000154|0|ect slyly against the even, final packages. quickly regular pinto beans wake c|61.1069|-149.859|61.1069|-149.859|
-2151|58|O|124608.69|1996-11-11|3-MEDIUM|Clerk#000000996|0|c requests. ironic platelets cajole across the quickly fluffy deposits.|61.1635|-149.881|61.1635|-149.881|
-2176|104|F|87248.17|1992-11-10|1-URGENT|Clerk#000000195|0|s haggle regularly accor|61.1201|-149.89|61.1201|-149.89|
-2177|136|O|183493.42|1997-01-20|3-MEDIUM|Clerk#000000161|0|ove the blithely unusual packages cajole carefully fluffily special request|61.1902|-149.908|61.1902|-149.908|
-2178|8|O|79594.68|1996-12-12|3-MEDIUM|Clerk#000000656|0|thely according to the instructions. furious|61.2104|-149.857|61.2104|-149.857|
-2179|41|O|77487.09|1996-09-07|2-HIGH|Clerk#000000935|0|ounts alongside of the furiously unusual braids cajol|61.1771|-149.97|61.1771|-149.97|
-2180|76|O|208481.57|1996-09-14|4-NOT SPECIFIED|Clerk#000000650|0|xpress, unusual pains. furiously ironic excu|61.1859|-149.976|61.1859|-149.976|
-2181|76|O|100954.64|1995-09-13|3-MEDIUM|Clerk#000000814|0|y against the ironic, even|61.2171|-149.9|61.2171|-149.9|
-2182|23|F|116003.11|1994-04-05|2-HIGH|Clerk#000000071|0|ccounts. quickly bold deposits across the excuses sl|61.1162|-149.755|61.1162|-149.755|
-2183|113|O|49841.12|1996-06-22|1-URGENT|Clerk#000000287|0| among the express, ironic packages. slyly ironic platelets integrat|61.1381|-149.844|61.1381|-149.844|
-2208|68|P|245388.06|1995-05-01|4-NOT SPECIFIED|Clerk#000000900|0|symptotes wake slyly blithely unusual packages.|61.1775|-149.941|61.1775|-149.941|
-2209|91|F|129086.93|1992-07-10|2-HIGH|Clerk#000000056|0|er above the slyly silent requests. furiously reg|61.1938|-149.878|61.1938|-149.878|
-2210|32|F|31689.46|1992-01-16|2-HIGH|Clerk#000000941|0| believe carefully quickly express pinto beans. deposi|61.1571|-149.883|61.1571|-149.883|
-2211|92|F|140031.23|1994-06-30|2-HIGH|Clerk#000000464|0|ffily bold courts e|61.1541|-149.958|61.1541|-149.958|
-2212|118|F|17231.05|1994-03-23|3-MEDIUM|Clerk#000000954|0|structions above the unusual requests use fur|61.135|-149.88|61.135|-149.88|
-2213|122|F|146136.10|1993-01-15|4-NOT SPECIFIED|Clerk#000000598|0|osits are carefully reg|61.1101|-149.857|61.1101|-149.857|
-2214|115|O|150345.63|1998-05-05|3-MEDIUM|Clerk#000000253|0|packages. fluffily even accounts haggle blithely. carefully ironic depen|61.1101|-149.857|61.1101|-149.857|
-2215|40|O|108239.46|1996-06-16|4-NOT SPECIFIED|Clerk#000000817|0|le final, final foxes. quickly regular gifts are carefully deposit|61.1101|-149.857|61.1101|-149.857|
-2240|56|F|174090.30|1992-03-06|4-NOT SPECIFIED|Clerk#000000622|0|accounts against the slyly express foxes are after the slyly regular |61.1101|-149.857|61.1101|-149.857|
-2241|103|F|165219.08|1993-05-11|1-URGENT|Clerk#000000081|0|y about the silent excuses. furiously ironic instructions along the sil|61.1101|-149.857|61.1101|-149.857|
-2242|82|O|15082.82|1997-07-20|4-NOT SPECIFIED|Clerk#000000360|0| pending multipliers. carefully express asymptotes use quickl|61.1101|-149.857|61.1101|-149.857|
-2243|49|O|10451.97|1995-06-10|2-HIGH|Clerk#000000813|0|ously regular deposits integrate s|61.1101|-149.857|61.1101|-149.857|
-2244|127|F|21207.08|1993-01-09|1-URGENT|Clerk#000001000|0|ckages. ironic, ironic accounts haggle blithely express excuses. |61.1101|-149.857|61.1101|-149.857|
-2245|58|F|150585.73|1993-04-28|3-MEDIUM|Clerk#000000528|0|ake carefully. braids haggle slyly quickly b|61.1101|-149.857|61.1101|-149.857|
-2246|113|O|85755.84|1996-05-27|4-NOT SPECIFIED|Clerk#000000739|0| final gifts sleep |61.1101|-149.857|61.1101|-149.857|
-2247|95|F|13491.31|1992-08-02|4-NOT SPECIFIED|Clerk#000000947|0|furiously regular packages. final brai|61.1101|-149.857|61.1101|-149.857|
-2272|139|F|127934.71|1993-04-13|2-HIGH|Clerk#000000449|0|s. bold, ironic pinto beans wake. silently specia|61.1101|-149.857|61.1101|-149.857|
-2273|136|O|142291.79|1996-12-14|5-LOW|Clerk#000000155|0|uickly express foxes haggle quickly against|61.1101|-149.857|61.1101|-149.857|
-2274|104|F|58273.89|1993-09-04|4-NOT SPECIFIED|Clerk#000000258|0|nstructions try to hag|61.1101|-149.857|61.1101|-149.857|
-2275|149|F|37398.90|1992-10-22|4-NOT SPECIFIED|Clerk#000000206|0| furiously furious platelets. slyly final packa|61.1101|-149.857|61.1101|-149.857|
-2276|43|O|141159.63|1996-04-29|4-NOT SPECIFIED|Clerk#000000821|0|ecial requests. fox|61.1101|-149.857|61.1101|-149.857|
-2277|89|F|79270.23|1995-01-02|4-NOT SPECIFIED|Clerk#000000385|0|accounts cajole. even i|61.1101|-149.857|61.1101|-149.857|
-2278|142|O|101878.46|1998-04-25|3-MEDIUM|Clerk#000000186|0|r pinto beans integrate after the carefully even deposits. blit|61.1101|-149.857|61.1101|-149.857|
-2279|80|F|142322.33|1993-02-23|3-MEDIUM|Clerk#000000898|0|de of the quickly unusual instructio|61.2141|-149.864|61.2141|-149.864|
-2304|46|F|93769.28|1994-01-07|4-NOT SPECIFIED|Clerk#000000415|0|onic platelets. ironic packages haggle. packages nag doggedly according to|61.2171|-149.9|61.2171|-149.9|
-2305|43|F|122964.66|1993-01-26|2-HIGH|Clerk#000000440|0|ove the furiously even acco|61.2171|-149.9|61.2171|-149.9|
-2306|28|O|244704.23|1995-07-26|2-HIGH|Clerk#000000975|0| wake furiously requests. permanent requests affix. final packages caj|61.2171|-149.9|61.2171|-149.9|
-2307|106|F|59417.76|1993-06-29|5-LOW|Clerk#000000952|0|furiously even asymptotes? carefully regular accounts|61.2171|-149.9|61.2171|-149.9|
-2308|25|F|58546.02|1992-10-25|4-NOT SPECIFIED|Clerk#000000609|0|ts. slyly final depo|61.2171|-149.9|61.2171|-149.9|
-2309|100|O|146933.07|1995-09-04|5-LOW|Clerk#000000803|0|he carefully pending packages. fluffily stealthy foxes engage carefully|61.2171|-149.9|61.2171|-149.9|
-2310|31|O|82928.12|1996-09-20|5-LOW|Clerk#000000917|0|wake carefully. unusual instructions nag ironic, regular excuse|61.2171|-149.9|61.2171|-149.9|
-2311|73|P|153233.93|1995-05-02|2-HIGH|Clerk#000000761|0|ly pending asymptotes-- furiously bold excus|61.2171|-149.9|61.2171|-149.9|
-2336|142|O|22294.51|1996-01-07|4-NOT SPECIFIED|Clerk#000000902|0|c, final excuses sleep furiously among the even theodolites. f|61.2171|-149.9|61.2171|-149.9|
-2337|142|O|45704.96|1997-06-18|4-NOT SPECIFIED|Clerk#000000754|0| quickly. final accounts haggle. carefully final acco|61.2171|-149.9|61.2171|-149.9|
-2338|140|O|28155.92|1997-09-15|2-HIGH|Clerk#000000951|0|riously final dugouts. final, ironic packages wake express, ironic id|61.2171|-149.9|61.2171|-149.9|
-2339|109|F|63470.78|1993-12-15|5-LOW|Clerk#000000847|0| against the regular |61.2171|-149.9|61.2171|-149.9|
-2340|65|O|30778.78|1996-01-12|1-URGENT|Clerk#000000964|0|ter the deposits sleep according to the slyly regular packages. carefully |61.2171|-149.9|61.2171|-149.9|
-2341|82|F|55950.21|1993-05-30|5-LOW|Clerk#000000443|0|sts-- blithely bold dolphins through the deposits nag blithely carefully re|61.2171|-149.9|61.2171|-149.9|
-2342|37|O|104038.78|1996-06-09|1-URGENT|Clerk#000000615|0|oost carefully across the regular accounts. blithely final d|61.2171|-149.9|61.2171|-149.9|
-2343|73|O|85381.00|1995-08-21|3-MEDIUM|Clerk#000000170|0|fluffily over the slyly special deposits. quickl|64.8487|-147.704|64.8487|-147.704|
-2368|13|F|101240.96|1993-08-20|1-URGENT|Clerk#000000830|0|t the bold instructions. carefully unusual |64.8486|-147.705|64.8486|-147.705|
-2369|110|O|73517.91|1996-12-24|2-HIGH|Clerk#000000752|0|iously even requests are dogged, express |64.8087|-147.71|64.8087|-147.71|
-2370|142|F|73924.21|1994-01-17|1-URGENT|Clerk#000000231|0|lyly final packages. quickly final deposits haggl|64.8363|-147.758|64.8363|-147.758|
-2371|19|O|193857.67|1998-01-07|1-URGENT|Clerk#000000028|0|ckages haggle at th|64.8476|-147.704|64.8476|-147.704|
-2372|31|O|104927.66|1997-11-21|5-LOW|Clerk#000000342|0|s: deposits haggle along the final ideas. careful|64.8302|-147.744|64.8302|-147.744|
-2373|28|F|55211.04|1994-03-12|4-NOT SPECIFIED|Clerk#000000306|0| even, special courts grow quickly. pending,|64.8476|-147.812|64.8476|-147.812|
-2374|4|F|115219.88|1993-10-29|4-NOT SPECIFIED|Clerk#000000081|0| blithely regular packages. blithely unusua|64.8144|-147.756|64.8144|-147.756|
-2375|5|O|106612.48|1996-11-20|3-MEDIUM|Clerk#000000197|0|unusual, pending theodolites cajole carefully |64.8183|-147.778|64.8183|-147.778|
-2400|37|O|92798.66|1998-07-25|5-LOW|Clerk#000000782|0|nusual courts nag against the carefully unusual pinto b|64.8494|-147.818|64.8494|-147.818|
-2401|148|O|88448.24|1997-07-29|4-NOT SPECIFIED|Clerk#000000531|0|ully unusual instructions boost carefully silently regular requests. |64.849|-147.822|64.849|-147.822|
-2402|67|O|70403.62|1996-09-06|4-NOT SPECIFIED|Clerk#000000162|0|slyly final sheaves sleep slyly. q|64.8367|-147.716|64.8367|-147.716|
-2403|55|O|111020.79|1998-04-11|3-MEDIUM|Clerk#000000820|0|furiously regular deposits use. furiously unusual accounts wake along the |64.8127|-147.772|64.8127|-147.772|
-2404|77|O|109077.69|1997-03-13|4-NOT SPECIFIED|Clerk#000000409|0|deposits breach furiously. ironic foxes haggle carefully bold packag|64.8143|-147.751|64.8143|-147.751|
-2405|73|O|115929.14|1996-12-23|3-MEDIUM|Clerk#000000535|0|ular, regular asympto|64.842|-147.721|64.842|-147.721|
-2406|7|O|182516.77|1996-10-28|5-LOW|Clerk#000000561|0|blithely regular accounts u|64.8403|-147.714|64.8403|-147.714|
-2407|55|O|112843.52|1998-06-19|2-HIGH|Clerk#000000068|0|uests affix slyly among the slyly regular depos|64.8371|-147.881|64.8371|-147.881|
-2432|103|O|62661.93|1996-07-13|1-URGENT|Clerk#000000115|0|re. slyly even deposits wake bra|64.8151|-147.707|64.8151|-147.707|
-2433|31|F|147071.86|1994-08-22|4-NOT SPECIFIED|Clerk#000000324|0|ess patterns are slyly. packages haggle carefu|64.8151|-147.707|64.8151|-147.707|
-2434|25|O|123956.25|1997-04-27|3-MEDIUM|Clerk#000000190|0|s. quickly ironic dolphins impress final deposits. blithel|64.8541|-147.81|64.8541|-147.81|
-2435|73|F|122490.66|1993-02-21|5-LOW|Clerk#000000112|0|es are carefully along the carefully final instructions. pe|64.8878|-147.496|64.8878|-147.496|
-2436|125|O|73990.08|1995-09-11|4-NOT SPECIFIED|Clerk#000000549|0|arefully. blithely bold deposits affix special accounts. final foxes nag. spe|64.8299|-147.728|64.8299|-147.728|
-2437|85|F|143411.69|1993-04-21|4-NOT SPECIFIED|Clerk#000000578|0|. theodolites wake slyly-- ironic, pending platelets above the carefully exp|64.8132|-147.762|64.8132|-147.762|
-2438|13|F|214494.39|1993-07-15|2-HIGH|Clerk#000000744|0|the final, regular warhorses. regularly |64.8372|-147.713|64.8372|-147.713|
-2439|55|O|41811.12|1997-03-15|2-HIGH|Clerk#000000819|0|lithely after the car|64.7927|-148.036|64.7927|-148.036|
-2464|145|O|30495.65|1997-11-23|5-LOW|Clerk#000000633|0|le about the instructions. courts wake carefully even|64.8717|-147.819|64.8717|-147.819|
-2465|34|O|180737.75|1995-06-24|1-URGENT|Clerk#000000078|0|al pinto beans. final, bold packages wake quickly|64.8527|-147.686|64.8527|-147.686|
-2466|19|F|161625.50|1994-03-06|1-URGENT|Clerk#000000424|0|c pinto beans. express deposits wake quickly. even, final courts nag. package|64.8371|-147.811|64.8371|-147.811|
-2467|35|O|7231.91|1995-07-16|4-NOT SPECIFIED|Clerk#000000914|0|pades sleep furiously. sometimes regular packages again|64.846|-147.705|64.846|-147.705|
-2468|112|O|160627.01|1997-06-09|4-NOT SPECIFIED|Clerk#000000260|0|ickly regular packages. slyly ruthless requests snooze quickly blithe|64.9064|-147.726|64.9064|-147.726|
-2469|124|O|192074.23|1996-11-26|5-LOW|Clerk#000000730|0| sleep closely regular instructions. furiously ironic instructi|64.9347|-147.56|64.9347|-147.56|
-2470|58|O|104966.33|1997-04-19|3-MEDIUM|Clerk#000000452|0|to the furiously final packages? pa|64.8861|-147.677|64.8861|-147.677|
-2471|89|O|34936.31|1998-03-12|4-NOT SPECIFIED|Clerk#000000860|0|carefully blithely regular pac|64.8302|-147.744|64.8302|-147.744|
-2496|136|F|140390.60|1994-01-09|2-HIGH|Clerk#000000142|0|slyly. pending instructions sleep. quic|60.6673|-151.311|60.6673|-151.311|
-2497|47|F|171326.48|1992-08-27|1-URGENT|Clerk#000000977|0|ily ironic pinto beans. furiously final platelets alongside of t|60.6997|-151.38|60.6997|-151.38|
-2498|97|F|45514.27|1993-11-08|5-LOW|Clerk#000000373|0|g the slyly special pinto beans. |60.5658|-151.244|60.5658|-151.244|
-2499|121|O|147243.86|1995-09-24|1-URGENT|Clerk#000000277|0|r the quickly bold foxes. bold instructi|60.6331|-151.163|60.6331|-151.163|
-2500|133|F|131122.82|1992-08-15|2-HIGH|Clerk#000000447|0|integrate slyly pending deposits. furiously ironic accounts across the s|60.6331|-151.163|60.6331|-151.163|
-2501|67|O|79380.51|1997-05-25|5-LOW|Clerk#000000144|0|ickly special theodolite|60.6331|-151.163|60.6331|-151.163|
-2502|70|F|33470.40|1993-05-28|4-NOT SPECIFIED|Clerk#000000914|0|lyly: carefully pending ideas affix again|60.6201|-151.332|60.6201|-151.332|
-2503|7|F|183671.08|1993-06-20|3-MEDIUM|Clerk#000000294|0|ly even packages was. ironic, regular deposits unwind furiously across the p|60.5004|-151.276|60.5004|-151.276|
-2528|55|F|92069.62|1994-11-20|1-URGENT|Clerk#000000789|0|ular dependencies? regular frays kindle according to the blith|60.6331|-151.163|60.6331|-151.163|
-2529|136|O|4104.30|1996-08-20|2-HIGH|Clerk#000000511|0|posits across the silent instructions wake blithely across |60.6331|-151.163|60.6331|-151.163|
-2530|128|F|58853.11|1994-03-21|3-MEDIUM|Clerk#000000291|0|ular instructions about the quic|60.6901|-151.321|60.6901|-151.321|
-2531|44|O|143212.85|1996-05-06|4-NOT SPECIFIED|Clerk#000000095|0|even accounts. furiously ironic excuses sleep fluffily. carefully silen|60.6676|-151.29|60.6676|-151.29|
-2532|94|O|116093.49|1995-10-11|2-HIGH|Clerk#000000498|0|the blithely pending accounts. regular, regular excuses boost aro|60.6331|-151.163|60.6331|-151.163|
-2533|50|O|168495.03|1997-03-24|1-URGENT|Clerk#000000594|0|ecial instructions. spec|60.5632|-151.266|60.5632|-151.266|
-2534|76|O|202784.54|1996-07-17|3-MEDIUM|Clerk#000000332|0|packages cajole ironic requests. furiously regular|60.6331|-151.163|60.6331|-151.163|
-2535|121|F|67018.30|1993-05-25|5-LOW|Clerk#000000296|0|phins cajole beneath the fluffily express asymptotes. c|60.6331|-151.163|60.6331|-151.163|
-2560|131|F|153426.79|1992-09-05|1-URGENT|Clerk#000000538|0|atelets; quickly sly requests|60.6509|-151.342|60.6509|-151.342|
-2561|58|O|137473.58|1997-11-14|1-URGENT|Clerk#000000861|0|ual requests. unusual deposits cajole furiously pending, regular platelets. |60.5601|-151.107|60.5601|-151.107|
-2562|10|F|136360.37|1992-08-01|1-URGENT|Clerk#000000467|0|elets. pending dolphins promise slyly. bo|60.5123|-151.275|60.5123|-151.275|
-2563|62|F|168952.10|1993-11-19|4-NOT SPECIFIED|Clerk#000000150|0|sly even packages after the furio|60.6076|-151.325|60.6076|-151.325|
-2564|77|F|3967.47|1994-09-09|2-HIGH|Clerk#000000718|0|usly regular pinto beans. orbits wake carefully. slyly e|60.6331|-151.163|60.6331|-151.163|
-2565|56|O|204438.57|1998-02-28|3-MEDIUM|Clerk#000000032|0|x-ray blithely along|60.5175|-151.235|60.5175|-151.235|
-2566|86|F|89992.48|1992-10-10|3-MEDIUM|Clerk#000000414|0|ructions boost bold ideas. idly ironic accounts use according to th|60.5535|-151.108|60.5535|-151.108|
-2567|70|O|263411.29|1998-02-27|2-HIGH|Clerk#000000031|0|detect. furiously ironic requests|60.5614|-151.275|60.5614|-151.275|
-2592|101|F|8225.96|1993-03-05|4-NOT SPECIFIED|Clerk#000000524|0|ts nag fluffily. quickly stealthy theodolite|60.5647|-151.195|60.5647|-151.195|
-2593|92|F|134726.09|1993-09-04|2-HIGH|Clerk#000000468|0|r the carefully final|60.6331|-151.163|60.6331|-151.163|
-2594|79|F|94866.39|1992-12-17|1-URGENT|Clerk#000000550|0|ests. theodolites above the blithely even accounts detect furio|60.6331|-151.163|60.6331|-151.163|
-2595|74|O|173130.20|1995-12-14|4-NOT SPECIFIED|Clerk#000000222|0|arefully ironic requests nag carefully ideas. |60.6331|-151.163|60.6331|-151.163|
-2596|43|O|74940.13|1996-08-17|1-URGENT|Clerk#000000242|0|requests. ironic, bold theodolites wak|60.6331|-151.163|60.6331|-151.163|
-2597|104|F|21964.66|1993-02-04|2-HIGH|Clerk#000000757|0|iously ruthless exc|60.6331|-151.163|60.6331|-151.163|
-2598|112|O|84871.50|1996-03-05|3-MEDIUM|Clerk#000000391|0| ironic notornis according to the blithely final requests should |60.6678|-151.31|60.6678|-151.31|
-2599|149|O|62807.13|1996-11-07|2-HIGH|Clerk#000000722|0|ts. slyly regular theodolites wake sil|60.5003|-151.276|60.5003|-151.276|
-2624|52|O|27148.63|1996-11-28|5-LOW|Clerk#000000930|0|ic, regular packages|60.6331|-151.163|60.6331|-151.163|
-2625|40|F|39382.74|1992-10-14|4-NOT SPECIFIED|Clerk#000000386|0| final deposits. blithely ironic ideas |61.5855|-149.326|61.5855|-149.326|
-2626|139|O|84314.51|1995-09-08|4-NOT SPECIFIED|Clerk#000000289|0|gside of the carefully special packages are furiously after the slyly express |61.5979|-149.437|61.5979|-149.437|
-2627|149|F|26798.65|1992-03-24|3-MEDIUM|Clerk#000000181|0|s. silent, ruthless requests|61.6141|-149.457|61.6141|-149.457|
-2628|56|F|165655.99|1993-10-22|5-LOW|Clerk#000000836|0|ajole across the blithely careful accounts. blithely silent deposits sl|61.5799|-149.461|61.5799|-149.461|
-2629|139|O|96458.03|1998-04-06|5-LOW|Clerk#000000680|0|uches dazzle carefully even, express excuses. ac|61.5845|-149.337|61.5845|-149.337|
-2630|85|F|127132.51|1992-10-24|5-LOW|Clerk#000000712|0|inal theodolites. ironic instructions s|61.5351|-149.558|61.5351|-149.558|
-2631|37|F|63103.32|1993-09-24|5-LOW|Clerk#000000833|0| quickly unusual deposits doubt around |61.5811|-149.45|61.5811|-149.45|
-2656|77|F|105492.37|1993-05-04|1-URGENT|Clerk#000000307|0|elets. slyly final accou|61.5793|-149.442|61.5793|-149.442|
-2657|25|O|148176.06|1995-10-17|2-HIGH|Clerk#000000160|0| foxes-- slyly final dependencies around the slyly final theodo|61.5661|-149.313|61.5661|-149.313|
-2658|14|O|163834.46|1995-09-23|3-MEDIUM|Clerk#000000400|0|bout the slyly regular accounts. ironic, |61.6141|-149.457|61.6141|-149.457|
-2659|83|F|79785.52|1993-12-18|4-NOT SPECIFIED|Clerk#000000758|0|cross the pending requests maintain |61.5786|-149.332|61.5786|-149.332|
-2660|127|O|16922.51|1995-08-05|5-LOW|Clerk#000000480|0|ly finally regular deposits. ironic theodolites cajole|61.5811|-149.45|61.5811|-149.45|
-2661|74|O|106036.84|1997-01-04|3-MEDIUM|Clerk#000000217|0|al, regular pinto beans. silently final deposits should have t|61.5825|-149.429|61.5825|-149.429|
-2662|37|O|87689.88|1996-08-21|3-MEDIUM|Clerk#000000589|0|bold pinto beans above the slyly final accounts affix furiously deposits. pac|61.6141|-149.457|61.6141|-149.457|
-2663|95|O|35131.80|1995-09-06|1-URGENT|Clerk#000000950|0|ar requests. furiously final dolphins along the fluffily spe|61.5531|-149.651|61.5531|-149.651|
-2688|98|F|181077.36|1992-01-24|2-HIGH|Clerk#000000720|0|have to nag according to the pending theodolites. sly|61.5531|-149.651|61.5531|-149.651|
-2689|103|F|41552.78|1992-04-09|4-NOT SPECIFIED|Clerk#000000698|0|press pains wake. furiously express theodolites alongsid|61.5698|-149.62|61.5698|-149.62|
-2690|94|O|224674.27|1996-03-31|3-MEDIUM|Clerk#000000760|0|ravely even theodolites |61.6141|-149.457|61.6141|-149.457|
-2691|7|F|30137.17|1992-04-30|5-LOW|Clerk#000000439|0|es at the regular deposits sleep slyly by the fluffy requests. eve|61.5474|-149.458|61.5474|-149.458|
-2692|62|O|24265.24|1997-12-02|3-MEDIUM|Clerk#000000878|0|es. regular asymptotes cajole above t|61.5825|-149.429|61.5825|-149.429|
-2693|19|O|66158.13|1996-09-04|1-URGENT|Clerk#000000370|0|ndle never. blithely regular packages nag carefully enticing platelets. ca|61.5955|-149.423|61.5955|-149.423|
-2694|121|O|102807.59|1996-03-14|5-LOW|Clerk#000000722|0| requests. bold deposits above the theodol|61.5801|-149.461|61.5801|-149.461|
-2695|58|O|138584.20|1996-08-20|1-URGENT|Clerk#000000697|0|ven deposits around the quickly regular packa|61.5785|-149.415|61.5785|-149.415|
-2720|31|F|161307.05|1993-06-08|1-URGENT|Clerk#000000948|0|quickly. special asymptotes are fluffily ironi|61.6402|-149.34|61.6402|-149.34|
-2721|79|O|59180.25|1996-01-27|2-HIGH|Clerk#000000401|0| ideas eat even, unusual ideas. theodolites are carefully|61.583|-149.457|61.583|-149.457|
-2722|35|F|50328.84|1994-04-09|5-LOW|Clerk#000000638|0|rding to the carefully quick deposits. bli|61.5907|-149.295|61.5907|-149.295|
-2723|61|O|104759.25|1995-10-06|5-LOW|Clerk#000000836|0|nts must have to cajo|61.6141|-149.457|61.6141|-149.457|
-2724|137|F|116069.66|1994-09-14|2-HIGH|Clerk#000000217|0| sleep blithely. blithely idle |61.5933|-149.397|61.5933|-149.397|
-2725|89|F|75144.68|1994-05-21|4-NOT SPECIFIED|Clerk#000000835|0|ular deposits. spec|61.6091|-149.77|61.6091|-149.77|
-2726|7|F|47753.00|1992-11-27|5-LOW|Clerk#000000470|0| blithely even dinos sleep care|61.577|-149.411|61.577|-149.411|
-2727|74|O|3089.42|1998-04-19|4-NOT SPECIFIED|Clerk#000000879|0|sual theodolites cajole enticingly above the furiously fin|61.6078|-149.322|61.6078|-149.322|
-2752|59|F|187932.30|1993-11-19|2-HIGH|Clerk#000000648|0| carefully regular foxes are quickly quickl|61.6131|-149.397|61.6131|-149.397|
-2753|16|F|159720.39|1993-11-30|2-HIGH|Clerk#000000380|0|ending instructions. unusual deposits|61.6648|-149.372|61.6648|-149.372|
-2754|145|F|25985.52|1994-04-03|2-HIGH|Clerk#000000960|0|cies detect slyly. |61.5531|-149.651|61.5531|-149.651|
-2755|118|F|101202.18|1992-02-07|4-NOT SPECIFIED|Clerk#000000177|0|ously according to the sly foxes. blithely regular pinto bean|61.5811|-149.45|61.5811|-149.45|
-2756|118|F|142323.38|1994-04-18|1-URGENT|Clerk#000000537|0|arefully special warho|61.583|-149.457|61.583|-149.457|
-2757|76|O|89792.48|1995-07-20|2-HIGH|Clerk#000000216|0| regular requests subl|61.1955|-149.9|61.1955|-149.9|
-2758|43|O|36671.88|1998-07-12|5-LOW|Clerk#000000863|0|s cajole according to the carefully special |61.1844|-149.897|61.1844|-149.897|
-2759|116|F|89731.10|1993-11-25|4-NOT SPECIFIED|Clerk#000000071|0|ts. regular, pending pinto beans sleep ab|61.1901|-149.892|61.1901|-149.892|
-2784|95|O|106635.21|1998-01-07|1-URGENT|Clerk#000000540|0|g deposits alongside of the silent requests s|61.1444|-149.867|61.1444|-149.867|
-2785|148|O|132854.79|1995-07-21|2-HIGH|Clerk#000000098|0|iously pending packages sleep according to the blithely unusual foxe|61.1955|-149.9|61.1955|-149.9|
-2786|79|F|178254.66|1992-03-22|2-HIGH|Clerk#000000976|0|al platelets cajole blithely ironic requests. ironic re|61.1893|-149.887|61.1893|-149.887|
-2787|103|O|3726.14|1995-09-30|1-URGENT|Clerk#000000906|0|he ironic, regular |61.2174|-149.888|61.2174|-149.888|
-2788|124|F|17172.66|1994-09-22|1-URGENT|Clerk#000000641|0|nts wake across the fluffily bold accoun|61.2227|-149.842|61.2227|-149.842|
-2789|37|O|219123.27|1998-03-14|2-HIGH|Clerk#000000972|0|gular patterns boost. carefully even re|61.1263|-149.872|61.1263|-149.872|
-2790|25|F|177458.97|1994-08-19|2-HIGH|Clerk#000000679|0| the carefully express deposits sleep slyly |61.1138|-149.866|61.1138|-149.866|
-2791|121|F|156697.55|1994-10-10|2-HIGH|Clerk#000000662|0|as. slyly ironic accounts play furiously bl|61.2157|-149.821|61.2157|-149.821|
-2816|58|F|42225.53|1994-09-20|2-HIGH|Clerk#000000289|0|kages at the final deposits cajole furious foxes. quickly |61.2174|-149.888|61.2174|-149.888|
-2817|40|F|71453.85|1994-04-19|3-MEDIUM|Clerk#000000982|0|ic foxes haggle upon the daringly even pinto beans. slyly|61.1855|-149.868|61.1855|-149.868|
-2818|49|F|120086.84|1994-12-12|3-MEDIUM|Clerk#000000413|0|eep furiously special ideas. express |61.1951|-149.873|61.1951|-149.873|
-2819|103|F|66927.16|1994-05-05|1-URGENT|Clerk#000000769|0|ngside of the blithely ironic dolphins. furio|61.1444|-149.867|61.1444|-149.867|
-2820|19|F|143813.39|1994-05-20|3-MEDIUM|Clerk#000000807|0|equests are furiously. carefu|61.1883|-149.735|61.1883|-149.735|
-2821|118|F|36592.48|1993-08-09|3-MEDIUM|Clerk#000000323|0|ng requests. even instructions are quickly express, silent instructi|61.2161|-149.876|61.2161|-149.876|
-2822|79|F|40142.15|1993-07-26|2-HIGH|Clerk#000000510|0|furiously against the accounts. unusual accounts aft|61.2161|-149.876|61.2161|-149.876|
-2823|79|O|171894.45|1995-09-09|2-HIGH|Clerk#000000567|0|encies. carefully fluffy accounts m|61.1893|-149.888|61.1893|-149.888|
-2848|70|F|116258.53|1992-03-10|1-URGENT|Clerk#000000256|0|ly fluffy foxes sleep furiously across the slyly regu|61.2174|-149.888|61.2174|-149.888|
-2849|46|O|180054.29|1996-04-30|2-HIGH|Clerk#000000659|0|al packages are after the quickly bold requests. carefully special |61.1914|-149.886|61.1914|-149.886|
-2850|100|O|122969.79|1996-10-02|2-HIGH|Clerk#000000392|0|, regular deposits. furiously pending packages hinder carefully carefully u|61.1541|-149.958|61.1541|-149.958|
-2851|145|O|7859.36|1997-09-07|5-LOW|Clerk#000000566|0|Tiresias wake quickly quickly even|61.1259|-149.717|61.1259|-149.717|
-2852|91|F|99050.81|1993-01-16|1-URGENT|Clerk#000000740|0|ruthless deposits against the final instructions use quickly al|61.2193|-149.902|61.2193|-149.902|
-2853|94|F|103641.15|1994-05-05|2-HIGH|Clerk#000000878|0|the carefully even packages.|61.1879|-149.886|61.1879|-149.886|
-2854|139|F|153568.02|1994-06-27|1-URGENT|Clerk#000000010|0| furiously ironic tithes use furiously |61.1372|-149.912|61.1372|-149.912|
-2855|49|F|48419.58|1993-04-04|4-NOT SPECIFIED|Clerk#000000973|0| silent, regular packages sleep |61.1101|-149.857|61.1101|-149.857|
-2880|8|F|145761.99|1992-03-15|2-HIGH|Clerk#000000756|0|ves maintain doggedly spec|61.1791|-149.94|61.1791|-149.94|
-2881|100|F|45695.84|1992-05-10|5-LOW|Clerk#000000864|0|uriously. slyly express requests according to the silent dol|61.2031|-149.749|61.2031|-149.749|
-2882|121|O|172872.37|1995-08-22|2-HIGH|Clerk#000000891|0|pending deposits. carefully eve|61.1914|-149.877|61.1914|-149.877|
-2883|121|F|170360.27|1995-01-23|5-LOW|Clerk#000000180|0|uses. carefully ironic accounts lose fluffil|61.1944|-149.883|61.1944|-149.883|
-2884|92|O|71683.84|1997-10-12|3-MEDIUM|Clerk#000000780|0|efully express instructions sleep against|61.1923|-149.886|61.1923|-149.886|
-2885|7|F|146896.72|1992-09-19|4-NOT SPECIFIED|Clerk#000000280|0|ly sometimes special excuses. final requests are |61.2123|-149.854|61.2123|-149.854|
-2886|109|F|94527.23|1994-11-13|4-NOT SPECIFIED|Clerk#000000619|0|uctions. ironic packages sle|61.2161|-149.876|61.2161|-149.876|
-2887|109|O|28571.39|1997-05-26|5-LOW|Clerk#000000566|0|slyly even pinto beans. slyly bold epitaphs cajole blithely above t|61.2171|-149.9|61.2171|-149.9|
-2912|94|F|27727.52|1992-03-12|5-LOW|Clerk#000000186|0|jole blithely above the quickly regular packages. carefully regular pinto bean|61.1125|-149.861|61.1125|-149.861|
-2913|43|O|130702.19|1997-07-12|3-MEDIUM|Clerk#000000118|0|mptotes doubt furiously slyly regu|61.1419|-149.896|61.1419|-149.896|
-2914|109|F|60867.14|1993-03-03|3-MEDIUM|Clerk#000000543|0|he slyly regular theodolites are furiously sile|61.145|-149.878|61.145|-149.878|
-2915|94|F|96015.13|1994-03-31|5-LOW|Clerk#000000410|0|ld packages. bold deposits boost blithely. ironic, unusual theodoli|61.1044|-149.865|61.1044|-149.865|
-2916|8|O|20182.22|1995-12-27|2-HIGH|Clerk#000000681|0|ithely blithe deposits sleep beyond the|61.1444|-149.876|61.1444|-149.876|
-2917|91|O|100714.13|1997-12-09|4-NOT SPECIFIED|Clerk#000000061|0| special dugouts among the special deposi|61.1|-149.85|61.1|-149.85|
-2918|118|O|21760.09|1996-09-08|3-MEDIUM|Clerk#000000439|0|ular deposits across th|61.1105|-149.861|61.1105|-149.861|
-2919|53|F|137223.14|1993-12-10|2-HIGH|Clerk#000000209|0|es. pearls wake quietly slyly ironic instructions--|61.1286|-149.957|61.1286|-149.957|
-2944|14|O|146581.14|1997-09-24|4-NOT SPECIFIED|Clerk#000000740|0|deas. permanently special foxes haggle carefully ab|61.1201|-149.89|61.1201|-149.89|
-2945|29|O|223507.72|1996-01-03|2-HIGH|Clerk#000000499|0|ons are carefully toward the permanent, bold pinto beans. regu|61.112|-149.871|61.112|-149.871|
-2946|125|O|102226.59|1996-02-05|5-LOW|Clerk#000000329|0|g instructions about the regular accounts sleep carefully along the pen|61.1427|-149.864|61.1427|-149.864|
-2947|70|P|43360.95|1995-04-26|1-URGENT|Clerk#000000464|0|ronic accounts. accounts run furiously d|61.1212|-149.947|61.1212|-149.947|
-2948|44|F|100758.71|1994-08-23|5-LOW|Clerk#000000701|0| deposits according to the blithely pending |61.1228|-149.939|61.1228|-149.939|
-2949|137|F|94231.71|1994-04-12|2-HIGH|Clerk#000000184|0|y ironic accounts use. quickly blithe accou|61.1093|-149.871|61.1093|-149.871|
-2950|136|O|183620.33|1997-07-06|1-URGENT|Clerk#000000833|0| dolphins around the furiously |61.145|-149.878|61.145|-149.878|
-2951|74|O|125509.17|1996-02-06|2-HIGH|Clerk#000000680|0|gular deposits above the finally regular ideas integrate idly stealthil|61.1191|-149.871|61.1191|-149.871|
-2976|29|F|145768.47|1993-12-10|4-NOT SPECIFIED|Clerk#000000159|0|. furiously ironic asymptotes haggle ruthlessly silently regular r|61.1003|-149.856|61.1003|-149.856|
-2977|73|O|25170.88|1996-08-27|3-MEDIUM|Clerk#000000252|0|quickly special platelets are furio|61.1113|-149.872|61.1113|-149.872|
-2978|44|P|139542.14|1995-05-03|1-URGENT|Clerk#000000135|0|d. even platelets are. ironic dependencies cajole slow, e|61.1084|-149.861|61.1084|-149.861|
-2979|133|O|116789.98|1996-03-23|3-MEDIUM|Clerk#000000820|0|even, ironic foxes sleep along|61.144|-149.878|61.144|-149.878|
-2980|4|O|187514.11|1996-09-14|3-MEDIUM|Clerk#000000661|0|y quick pinto beans wake. slyly re|61.1426|-149.877|61.1426|-149.877|
-2981|49|O|37776.79|1998-07-29|5-LOW|Clerk#000000299|0|hely among the express foxes. blithely stealthy requests cajole boldly. regu|61.1173|-149.861|61.1173|-149.861|
-2982|85|F|55582.94|1995-03-19|2-HIGH|Clerk#000000402|0|lyly. express theodolites affix slyly after the slyly speci|61.1347|-149.914|61.1347|-149.914|
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index 5e7b5c9..30d47ea 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -1,96 +1,109 @@
-<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.examples</groupId>
-  <artifactId>hyracks-integration-tests</artifactId>
-  <name>hyracks-integration-tests</name>
-  <parent>
-    <groupId>edu.uci.ics.hyracks</groupId>
-    <artifactId>hyracks-examples</artifactId>
-    <version>0.2.3-SNAPSHOT</version>
-  </parent>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <version>2.0.2</version>
-        <configuration>
-          <source>1.7</source>
-          <target>1.7</target>
-          <fork>true</fork>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-  <dependencies>
-  	<dependency>
-  		<groupId>junit</groupId>
-  		<artifactId>junit</artifactId>
-  		<version>4.8.1</version>
-  		<type>jar</type>
-  		<scope>test</scope>
-  	</dependency>
-  	<dependency>
-        <groupId>edu.uci.ics.hyracks</groupId>
-        <artifactId>hyracks-client</artifactId>
-        <version>0.2.3-SNAPSHOT</version>
-        <type>jar</type>
-        <scope>compile</scope>
-    </dependency>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-control-cc</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-control-nc</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-storage-am-btree</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-storage-am-invertedindex</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-storage-am-rtree</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-test-support</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>test</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-data-std</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  	</dependency>
-  </dependencies>
+<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.examples</groupId>
+	<artifactId>hyracks-integration-tests</artifactId>
+	<name>hyracks-integration-tests</name>
+	<parent>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<artifactId>hyracks-examples</artifactId>
+		<version>0.2.3-SNAPSHOT</version>
+	</parent>
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>2.0.2</version>
+				<configuration>
+					<source>1.7</source>
+					<target>1.7</target>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+	<dependencies>
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>4.8.1</version>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-dataflow-std</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-control-cc</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-control-nc</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-btree</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-rtree</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-btree</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-test-support</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-data-std</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+		</dependency>
+  	    <dependency>
+            <groupId>edu.uci.ics.hyracks</groupId>
+            <artifactId>hyracks-client</artifactId>
+            <version>0.2.3-SNAPSHOT</version>
+            <type>jar</type>
+            <scope>compile</scope>
+        </dependency>
+	</dependencies>
 </project>
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
new file mode 100644
index 0000000..0af1193
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -0,0 +1,327 @@
+/*
+ * Copyright 2009-2010 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.tests.am.btree;
+
+import java.io.DataOutput;
+import java.io.File;
+
+import org.junit.After;
+import org.junit.Before;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+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.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+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.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.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.test.support.TestIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+
+public abstract class AbstractBTreeOperatorTest extends AbstractIntegrationTest {
+    static {
+        TestStorageManagerComponentHolder.init(8192, 20, 20);
+    }
+
+    protected final IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+    protected final IIndexLifecycleManagerProvider lcManagerProvider = new TestIndexLifecycleManagerProvider();
+    protected IIndexDataflowHelperFactory dataflowHelperFactory;
+
+    // field, type and key declarations for primary index
+    protected final int primaryFieldCount = 6;
+    protected final ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+    protected final int primaryKeyFieldCount = 1;
+    protected final IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+    protected final int[] primaryBloomFilterKeyFields = new int[primaryKeyFieldCount];
+
+    protected final RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+    // to be set by subclasses
+    protected String primaryFileName;
+    protected IFileSplitProvider primarySplitProvider;
+
+    // field, type and key declarations for secondary indexes
+    protected final int secondaryFieldCount = 2;
+    protected final ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+    protected final int secondaryKeyFieldCount = 2;
+    protected final IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
+    protected final int[] secondaryBloomFilterKeyFields = new int[secondaryKeyFieldCount];
+
+    protected final RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+    protected String secondaryFileName;
+    protected IFileSplitProvider secondarySplitProvider;
+
+    protected ITreeIndexOperatorTestHelper testHelper;
+
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new BTreeOperatorTestHelper();
+    }
+
+    @Before
+    public void setup() throws Exception {
+        testHelper = createTestHelper();
+        dataflowHelperFactory = createDataFlowHelperFactory();
+        primaryFileName = testHelper.getPrimaryIndexName();
+        primarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID, new FileReference(
+                new File(primaryFileName))) });
+        secondaryFileName = testHelper.getSecondaryIndexName();
+        secondarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                new FileReference(new File(secondaryFileName))) });
+
+        // field, type and key declarations for primary index
+        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+        primaryBloomFilterKeyFields[0] = 0;
+
+        // field, type and key declarations for secondary indexes
+        secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+        secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+        secondaryBloomFilterKeyFields[0] = 0;
+        secondaryBloomFilterKeyFields[1] = 1;
+    }
+
+    protected abstract IIndexDataflowHelperFactory createDataFlowHelperFactory();
+
+    public void createPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
+                lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                primaryBloomFilterKeyFields, dataflowHelperFactory, localResourceFactoryProvider,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
+        spec.addRoot(primaryCreateOp);
+        runTest(spec);
+    }
+
+    protected void loadPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/tpch0.001/orders-part1.tbl"))) };
+        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
+
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
+
+        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
+
+        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,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
+
+        spec.addRoot(primaryBtreeBulkLoad);
+        runTest(spec);
+    }
+
+    public void createSecondaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+        TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+                storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, secondaryBloomFilterKeyFields, dataflowHelperFactory,
+                localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryCreateOp, NC1_ID);
+        spec.addRoot(secondaryCreateOp);
+        runTest(spec);
+    }
+
+    protected void loadSecondaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build dummy tuple containing nothing
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] lowKeyFields = null; // - infinity
+        int[] highKeyFields = null; // + infinity
+
+        // scan primary index
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+
+        // sort based on secondary keys
+        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 3, 0 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, primaryRecDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
+
+        // load secondary index
+        int[] fieldPermutation = { 3, 0 };
+        TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+                storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, secondaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L,
+                dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, sorter, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, secondaryBtreeBulkLoad, 0);
+
+        spec.addRoot(secondaryBtreeBulkLoad);
+        runTest(spec);
+    }
+
+    protected void insertPipeline(boolean useUpsert) throws Exception {
+        IndexOperation pipelineOperation = useUpsert ? IndexOperation.UPSERT : IndexOperation.INSERT;
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/tpch0.001/orders-part2.tbl"))) };
+        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
+
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
+
+        // insert into primary index
+        int[] primaryFieldPermutation = { 0, 1, 2, 4, 5, 7 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor primaryBtreeInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, ordersDesc, storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, primaryBloomFilterKeyFields, primaryFieldPermutation, pipelineOperation,
+                dataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeInsertOp, NC1_ID);
+
+        // first secondary index
+        int[] fieldPermutationB = { 4, 0 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, ordersDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, secondaryBloomFilterKeyFields, fieldPermutationB, pipelineOperation,
+                dataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryInsertOp, NC1_ID);
+
+        NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, nullSink, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, primaryBtreeInsertOp, 0);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeInsertOp, 0, secondaryInsertOp, 0);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryInsertOp, 0, nullSink, 0);
+
+        spec.addRoot(nullSink);
+        runTest(spec);
+    }
+
+    protected void destroyPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IndexDropOperatorDescriptor primaryDropOp = new IndexDropOperatorDescriptor(spec, storageManager,
+                lcManagerProvider, primarySplitProvider, dataflowHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryDropOp, NC1_ID);
+        spec.addRoot(primaryDropOp);
+        runTest(spec);
+    }
+
+    protected void destroySecondaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IndexDropOperatorDescriptor secondaryDropOp = new IndexDropOperatorDescriptor(spec, storageManager,
+                lcManagerProvider, secondarySplitProvider, dataflowHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryDropOp, NC1_ID);
+        spec.addRoot(secondaryDropOp);
+        runTest(spec);
+    }
+
+    @After
+    public abstract void cleanup() throws Exception;
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeOperatorTestHelper.java
new file mode 100644
index 0000000..337bfda
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeOperatorTestHelper.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2009-2010 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.tests.am.btree;
+
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.tests.am.common.TreeOperatorTestHelper;
+
+public class BTreeOperatorTestHelper extends TreeOperatorTestHelper {
+
+    public IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return new BTreeDataflowHelperFactory();
+    }
+
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
new file mode 100644
index 0000000..7e1c42a
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2010 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.tests.am.btree;
+
+import java.io.DataOutput;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+
+public class BTreePrimaryIndexScanOperatorTest extends AbstractBTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+    }
+
+    @Test
+    public void scanPrimaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build dummy tuple containing nothing
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] lowKeyFields = null; // - infinity
+        int[] highKeyFields = null; // + infinity
+
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
new file mode 100644
index 0000000..64bc657
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2009-2010 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.tests.am.btree;
+
+import java.io.DataOutput;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+
+public class BTreePrimaryIndexSearchOperatorTest extends AbstractBTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+    }
+
+    @Test
+    public void searchPrimaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build tuple containing low and high search key
+        // high key and low key
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        // low key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("100", dos);
+        tb.addFieldEndOffset();
+        // high key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("200", dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] lowKeyFields = { 0 };
+        int[] highKeyFields = { 1 };
+
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java
new file mode 100644
index 0000000..c8d95ce
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 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.tests.am.btree;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexStatsOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+
+public class BTreePrimaryIndexStatsOperatorTest extends AbstractBTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+    }
+
+    @Test
+    public void showPrimaryIndexStats() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
+                lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                primaryBloomFilterKeyFields, dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
new file mode 100644
index 0000000..5a600a6
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2009-2010 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.tests.am.btree;
+
+import java.io.DataOutput;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+
+public class BTreeSecondaryIndexInsertOperatorTest extends AbstractBTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+        createSecondaryIndex();
+        loadSecondaryIndex();
+        insertPipeline(false);
+    }
+
+    @Test
+    public void searchUpdatedSecondaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build tuple containing search keys (only use the first key as search
+        // key)
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        // low key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
+        tb.addFieldEndOffset();
+        // high key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] secondaryLowKeyFields = { 0 };
+        int[] secondaryHighKeyFields = { 1 };
+
+        // search secondary index
+        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+                secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, secondaryBloomFilterKeyFields, secondaryLowKeyFields,
+                secondaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
+
+        // second field from the tuples coming from secondary index
+        int[] primaryLowKeyFields = { 1 };
+        // second field from the tuples coming from secondary index
+        int[] primaryHighKeyFields = { 1 };
+
+        // search primary index
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
+                dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+        destroySecondaryIndex();
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
new file mode 100644
index 0000000..e3005cf
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2009-2010 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.tests.am.btree;
+
+import java.io.DataOutput;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+
+public class BTreeSecondaryIndexSearchOperatorTest extends AbstractBTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+        createSecondaryIndex();
+        loadSecondaryIndex();
+    }
+
+    @Test
+    public void searchSecondaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build tuple containing search keys (only use the first key as search
+        // key)
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        // low key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
+        tb.addFieldEndOffset();
+        // high key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] secondaryLowKeyFields = { 0 };
+        int[] secondaryHighKeyFields = { 1 };
+
+        // search secondary index
+        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+                secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, secondaryBloomFilterKeyFields, secondaryLowKeyFields,
+                secondaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
+
+        int[] primaryLowKeyFields = { 1 }; // second field from the tuples
+        // coming from secondary index
+        int[] primaryHighKeyFields = { 1 }; // second field from the tuples
+        // coming from secondary index
+
+        // search primary index
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
+                dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+        destroySecondaryIndex();
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
new file mode 100644
index 0000000..758e926
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
@@ -0,0 +1,107 @@
+package edu.uci.ics.hyracks.tests.am.btree;
+
+import java.io.DataOutput;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+
+public class BTreeSecondaryIndexUpsertOperatorTest extends AbstractBTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+        createSecondaryIndex();
+        loadSecondaryIndex();
+        insertPipeline(true);
+    }
+
+    @Test
+    public void searchUpdatedSecondaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build tuple containing search keys (only use the first key as search
+        // key)
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        // low key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
+        tb.addFieldEndOffset();
+        // high key
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] secondaryLowKeyFields = { 0 };
+        int[] secondaryHighKeyFields = { 1 };
+
+        // search secondary index
+        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+                secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, secondaryBloomFilterKeyFields, secondaryLowKeyFields,
+                secondaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
+
+        // second field from the tuples coming from secondary index
+        int[] primaryLowKeyFields = { 1 };
+        // second field from the tuples coming from secondary index
+        int[] primaryHighKeyFields = { 1 };
+
+        // search primary index
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
+                dataflowHelperFactory, false, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+        destroySecondaryIndex();
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/ITreeIndexOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/ITreeIndexOperatorTestHelper.java
new file mode 100644
index 0000000..f0c4519
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/ITreeIndexOperatorTestHelper.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.tests.am.common;
+
+public interface ITreeIndexOperatorTestHelper {
+    public String getPrimaryIndexName();
+
+    public String getSecondaryIndexName();
+
+    public void cleanup(String primaryFileName, String secondaryFileName);
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/LSMTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/LSMTreeOperatorTestHelper.java
new file mode 100644
index 0000000..80b5285
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/LSMTreeOperatorTestHelper.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 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.tests.am.common;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.util.Date;
+
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+
+public class LSMTreeOperatorTestHelper extends TreeOperatorTestHelper {
+
+    protected final IOManager ioManager;
+
+    public LSMTreeOperatorTestHelper(IOManager ioManager) {
+        this.ioManager = ioManager;
+    }
+
+    public String getPrimaryIndexName() {
+        return "primary" + simpleDateFormat.format(new Date());
+    }
+
+    public String getSecondaryIndexName() {
+        return "secondary" + simpleDateFormat.format(new Date());
+    }
+
+    @Override
+    public void cleanup(String primaryFileName, String secondaryFileName) {
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            File primaryDir = new File(dev.getPath(), primaryFileName);
+            cleanupDir(primaryDir);
+            File secondaryDir = new File(dev.getPath(), secondaryFileName);
+            cleanupDir(secondaryDir);
+        }
+    }
+
+    private void cleanupDir(File dir) {
+        if (!dir.exists()) {
+            return;
+        }
+        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();
+            }
+        }
+        dir.delete();
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java
new file mode 100644
index 0000000..935724b
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 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.tests.am.common;
+
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+public class TreeOperatorTestHelper implements ITreeIndexOperatorTestHelper {
+
+    protected final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    protected final String sep = System.getProperty("file.separator");
+    protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
+    protected static int DEFAULT_MEM_NUM_PAGES = 1000;
+
+    public String getPrimaryIndexName() {
+        return System.getProperty("java.io.tmpdir") + sep + "primary" + simpleDateFormat.format(new Date());
+    }
+
+    public String getSecondaryIndexName() {
+        return System.getProperty("java.io.tmpdir") + sep + "secondary" + simpleDateFormat.format(new Date());
+    }
+
+    @Override
+    public void cleanup(String primaryFileName, String secondaryFileName) {
+        File primary = new File(primaryFileName);
+        if (primary.exists()) {
+            primary.deleteOnExit();
+        }
+        File secondary = new File(secondaryFileName);
+        if (secondary.exists()) {
+            secondary.deleteOnExit();
+        }
+    }
+}
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
new file mode 100644
index 0000000..808afac
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/AbstractfWordInvertedIndexTest.java
@@ -0,0 +1,346 @@
+/*
+ * Copyright 2009-2012 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.tests.am.invertedindex;
+
+import java.io.DataOutput;
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+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.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+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.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.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.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexCreateOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.test.support.TestIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
+import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+
+@SuppressWarnings("rawtypes")
+public abstract class AbstractfWordInvertedIndexTest extends AbstractIntegrationTest {
+    static {
+        TestStorageManagerComponentHolder.init(8192, 20, 20);
+    }
+
+    protected static final int MERGE_THRESHOLD = 3;
+
+    protected IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+    protected IIndexLifecycleManagerProvider lcManagerProvider = new TestIndexLifecycleManagerProvider();
+    protected IIndexDataflowHelperFactory btreeDataflowHelperFactory = new BTreeDataflowHelperFactory();
+    protected IIndexDataflowHelperFactory invertedIndexDataflowHelperFactory;
+
+    protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    protected final static String sep = System.getProperty("file.separator");
+    protected final String dateString = simpleDateFormat.format(new Date());
+    protected final String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree" + dateString;
+    protected final String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree" + dateString;
+
+    protected IFileSplitProvider primaryFileSplitProvider = new ConstantFileSplitProvider(
+            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
+    protected IFileSplitProvider btreeFileSplitProvider = new ConstantFileSplitProvider(
+            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(btreeFileName))) });
+
+    // Primary BTree index.
+    protected int primaryFieldCount = 2;
+    protected ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+    protected int primaryKeyFieldCount = 1;
+    protected IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+    protected RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+    // Inverted index BTree dictionary.
+    protected ITypeTraits[] tokenTypeTraits;
+    protected IBinaryComparatorFactory[] tokenComparatorFactories;
+
+    // Inverted index stuff.
+    protected int invListElementFieldCount = 1;
+    protected ITypeTraits[] invListsTypeTraits = new ITypeTraits[invListElementFieldCount];
+    protected IBinaryComparatorFactory[] invListsComparatorFactories = new IBinaryComparatorFactory[invListElementFieldCount];
+    protected RecordDescriptor invListsRecDesc = new RecordDescriptor(
+            new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+    protected RecordDescriptor tokenizerRecDesc;
+
+    // Tokenizer stuff.
+    protected ITokenFactory tokenFactory = new UTF8WordTokenFactory();
+    protected IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
+            tokenFactory);
+
+    // Sorting stuff.
+    IBinaryComparatorFactory[] sortComparatorFactories;
+
+    @Before
+    public void setup() throws Exception {
+        prepare();
+
+        // Field declarations and comparators for primary BTree index.
+        primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
+        // Field declarations and comparators for inverted lists.
+        invListsTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        invListsComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
+        createPrimaryIndex();
+        loadPrimaryIndex();
+        printPrimaryIndex();
+        createInvertedIndex();
+        loadInvertedIndex();
+    }
+
+    protected abstract void prepare();
+
+    protected abstract boolean addNumTokensKey();
+
+    public void createPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
+                lcManagerProvider, primaryFileSplitProvider, primaryTypeTraits, primaryComparatorFactories, null,
+                btreeDataflowHelperFactory, localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
+        spec.addRoot(primaryCreateOp);
+        runTest(spec);
+    }
+
+    public void createInvertedIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        ILocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+        LSMInvertedIndexCreateOperatorDescriptor invIndexCreateOp = new LSMInvertedIndexCreateOperatorDescriptor(spec,
+                storageManager, btreeFileSplitProvider, lcManagerProvider, tokenTypeTraits, tokenComparatorFactories,
+                invListsTypeTraits, invListsComparatorFactories, tokenizerFactory, invertedIndexDataflowHelperFactory,
+                localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexCreateOp, NC1_ID);
+        spec.addRoot(invIndexCreateOp);
+        runTest(spec);
+    }
+
+    @Test
+    public void testConjunctiveSearcher() throws Exception {
+        IInvertedIndexSearchModifierFactory conjunctiveSearchModifierFactory = new ConjunctiveSearchModifierFactory();
+        searchInvertedIndex("of", conjunctiveSearchModifierFactory);
+        searchInvertedIndex("3d", conjunctiveSearchModifierFactory);
+        searchInvertedIndex("of the human", conjunctiveSearchModifierFactory);
+    }
+
+    private IOperatorDescriptor createFileScanOp(JobSpecification spec) {
+        FileSplit[] dblpTitleFileSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/cleanednumbereddblptitles.txt"))) };
+        IFileSplitProvider dblpTitleSplitProvider = new ConstantFileSplitProvider(dblpTitleFileSplits);
+        RecordDescriptor dblpTitleRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+        FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dblpTitleScanner, NC1_ID);
+        return dblpTitleScanner;
+    }
+
+    private IOperatorDescriptor createPrimaryBulkLoadOp(JobSpecification spec) {
+        int[] fieldPermutation = { 0, 1 };
+        TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+                storageManager, lcManagerProvider, primaryFileSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, null, fieldPermutation, 0.7f, true, 1000L, btreeDataflowHelperFactory,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
+        return primaryBtreeBulkLoad;
+    }
+
+    private IOperatorDescriptor createScanKeyProviderOp(JobSpecification spec) throws HyracksDataException {
+        // build dummy tuple containing nothing
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+        DataOutput dos = tb.getDataOutput();
+        tb.reset();
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+        tb.addFieldEndOffset();
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+        return keyProviderOp;
+    }
+
+    private IOperatorDescriptor createPrimaryScanOp(JobSpecification spec) throws HyracksDataException {
+        int[] lowKeyFields = null; // - infinity
+        int[] highKeyFields = null; // + infinity
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primaryFileSplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, null, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory,
+                false, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+        return primaryBtreeSearchOp;
+    }
+
+    private void loadPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        // Assuming that the data is pre-sorted on the key. No need to sort
+        // before bulk load.
+        IOperatorDescriptor fileScanOp = createFileScanOp(spec);
+        IOperatorDescriptor primaryBulkLoad = createPrimaryBulkLoadOp(spec);
+        spec.connect(new OneToOneConnectorDescriptor(spec), fileScanOp, 0, primaryBulkLoad, 0);
+        spec.addRoot(primaryBulkLoad);
+        runTest(spec);
+    }
+
+    private void printPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
+        IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, printer, 0);
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    private IOperatorDescriptor createExternalSortOp(JobSpecification spec, int[] sortFields,
+            RecordDescriptor outputRecDesc) {
+        ExternalSortOperatorDescriptor externalSortOp = new ExternalSortOperatorDescriptor(spec, 1000, sortFields,
+                sortComparatorFactories, outputRecDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, externalSortOp, NC1_ID);
+        return externalSortOp;
+    }
+
+    private IOperatorDescriptor createBinaryTokenizerOp(JobSpecification spec, int docField, int[] keyFields) {
+        BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
+                tokenizerRecDesc, tokenizerFactory, docField, keyFields, addNumTokensKey());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
+        return binaryTokenizer;
+    }
+
+    private IOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec, int[] fieldPermutation) {
+        LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
+                spec, fieldPermutation, true, 1000L, storageManager, btreeFileSplitProvider, lcManagerProvider,
+                tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
+                tokenizerFactory, invertedIndexDataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexBulkLoadOp, NC1_ID);
+        return invIndexBulkLoadOp;
+    }
+
+    public void loadInvertedIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
+        IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
+        int docField = 1;
+        int[] keyFields = { 0 };
+        IOperatorDescriptor binaryTokenizerOp = createBinaryTokenizerOp(spec, docField, keyFields);
+        int[] sortFields = new int[sortComparatorFactories.length];
+        int[] fieldPermutation = new int[sortComparatorFactories.length];
+        for (int i = 0; i < sortFields.length; i++) {
+            sortFields[i] = i;
+            fieldPermutation[i] = i;
+        }
+        IOperatorDescriptor externalSortOp = createExternalSortOp(spec, sortFields, tokenizerRecDesc);
+        IOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec, fieldPermutation);
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, binaryTokenizerOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), binaryTokenizerOp, 0, externalSortOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), externalSortOp, 0, invIndexBulkLoadOp, 0);
+        spec.addRoot(invIndexBulkLoadOp);
+        runTest(spec);
+    }
+
+    private IOperatorDescriptor createQueryProviderOp(JobSpecification spec, String queryString)
+            throws HyracksDataException {
+        // Build tuple with exactly one field, which is the query,
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+        DataOutput dos = tb.getDataOutput();
+        tb.reset();
+        UTF8StringSerializerDeserializer.INSTANCE.serialize(queryString, dos);
+        tb.addFieldEndOffset();
+        ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
+        ConstantTupleSourceOperatorDescriptor queryProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                queryRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, queryProviderOp, NC1_ID);
+        return queryProviderOp;
+    }
+
+    private IOperatorDescriptor createInvertedIndexSearchOp(JobSpecification spec,
+            IInvertedIndexSearchModifierFactory searchModifierFactory) {
+        LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(spec,
+                0, storageManager, btreeFileSplitProvider, lcManagerProvider, tokenTypeTraits,
+                tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
+                invertedIndexDataflowHelperFactory, tokenizerFactory, searchModifierFactory, invListsRecDesc, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexSearchOp, NC1_ID);
+        return invIndexSearchOp;
+    }
+
+    public void searchInvertedIndex(String queryString, IInvertedIndexSearchModifierFactory searchModifierFactory)
+            throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IOperatorDescriptor queryProviderOp = createQueryProviderOp(spec, queryString);
+        IOperatorDescriptor invIndexSearchOp = createInvertedIndexSearchOp(spec, searchModifierFactory);
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        spec.connect(new OneToOneConnectorDescriptor(spec), queryProviderOp, 0, invIndexSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), invIndexSearchOp, 0, printer, 0);
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/BinaryTokenizerOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/BinaryTokenizerOperatorTest.java
new file mode 100644
index 0000000..47480da
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/BinaryTokenizerOperatorTest.java
@@ -0,0 +1,89 @@
+package edu.uci.ics.hyracks.tests.am.invertedindex;
+
+import java.io.File;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+
+public class BinaryTokenizerOperatorTest extends AbstractIntegrationTest {
+
+    @Test
+    public void tokenizerTest() throws Exception {
+        test(false);
+    }
+
+    @Test
+    public void tokenizerWithNumTokensTest() throws Exception {
+        test(true);
+    }
+
+    private void test(boolean addNumTokensKey) throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] dblpTitleFileSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/cleanednumbereddblptitles.txt"))) };
+        IFileSplitProvider dblpTitleSplitProvider = new ConstantFileSplitProvider(dblpTitleFileSplits);
+        RecordDescriptor dblpTitleRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+        FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dblpTitleScanner, NC1_ID);
+
+        RecordDescriptor tokenizerRecDesc;
+        if (!addNumTokensKey) {
+            tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                    UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+        } else {
+            tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                    UTF8StringSerializerDeserializer.INSTANCE, ShortSerializerDeserializer.INSTANCE,
+                    IntegerSerializerDeserializer.INSTANCE });
+        }
+
+        ITokenFactory tokenFactory = new UTF8WordTokenFactory();
+        IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
+                tokenFactory);
+        int[] keyFields = { 0 };
+        BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
+                tokenizerRecDesc, tokenizerFactory, 1, keyFields, addNumTokensKey);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), binaryTokenizer, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java
new file mode 100644
index 0000000..62d4362
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2012 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.tests.am.invertedindex;
+
+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.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+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.SynchronousSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
+
+public class PartitionedWordInvertedIndexTest extends AbstractfWordInvertedIndexTest {
+
+    @Override
+    protected void prepare() {
+        // Field declarations and comparators for tokens.
+        tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS, ShortPointable.TYPE_TRAITS };
+        tokenComparatorFactories = new IBinaryComparatorFactory[] {
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                PointableBinaryComparatorFactory.of(ShortPointable.FACTORY) };
+
+        tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, ShortSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE });
+
+        sortComparatorFactories = new IBinaryComparatorFactory[] {
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                PointableBinaryComparatorFactory.of(ShortPointable.FACTORY),
+                PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) };
+
+        invertedIndexDataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
+                new ConstantMergePolicyProvider(MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
+                SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE,
+                DEFAULT_MEM_NUM_PAGES);
+    }
+
+    @Override
+    protected boolean addNumTokensKey() {
+        return true;
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
new file mode 100644
index 0000000..c35c6c9
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2012 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.tests.am.invertedindex;
+
+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.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+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.SynchronousSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
+
+public class WordInvertedIndexTest extends AbstractfWordInvertedIndexTest {
+
+    @Override
+    protected void prepare() {
+        // Field declarations and comparators for tokens.
+        tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
+        tokenComparatorFactories = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
+                .of(UTF8StringPointable.FACTORY) };
+
+        tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+        sortComparatorFactories = new IBinaryComparatorFactory[] {
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) };
+
+        invertedIndexDataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(new ConstantMergePolicyProvider(
+                MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
+                SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES);
+    }
+
+    @Override
+    protected boolean addNumTokensKey() {
+        return false;
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
new file mode 100644
index 0000000..912ab0e
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.btree;
+
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+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.SynchronousSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
+
+public class LSMBTreeOperatorTestHelper extends LSMTreeOperatorTestHelper {
+
+    private static final int MERGE_THRESHOLD = 3;
+
+    public LSMBTreeOperatorTestHelper(IOManager ioManager) {
+        super(ioManager);
+    }
+
+    public IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return new LSMBTreeDataflowHelperFactory(new ConstantMergePolicyProvider(MERGE_THRESHOLD),
+                ThreadCountingOperationTrackerFactory.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
+                NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES);
+    }
+
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexScanOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexScanOperatorTest.java
new file mode 100644
index 0000000..d751399
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexScanOperatorTest.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.btree;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.tests.am.btree.BTreePrimaryIndexScanOperatorTest;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+
+public class LSMBTreePrimaryIndexScanOperatorTest extends BTreePrimaryIndexScanOperatorTest {
+
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new LSMBTreeOperatorTestHelper(TestStorageManagerComponentHolder.getIOManager());
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
new file mode 100644
index 0000000..bdcf3f6
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.btree;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.tests.am.btree.BTreePrimaryIndexSearchOperatorTest;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+
+public class LSMBTreePrimaryIndexSearchOperatorTest extends BTreePrimaryIndexSearchOperatorTest {
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new LSMBTreeOperatorTestHelper(TestStorageManagerComponentHolder.getIOManager());
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexInsertOperatorTest.java
new file mode 100644
index 0000000..5ba7279
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexInsertOperatorTest.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.btree;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.tests.am.btree.BTreeSecondaryIndexInsertOperatorTest;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+
+public class LSMBTreeSecondaryIndexInsertOperatorTest extends BTreeSecondaryIndexInsertOperatorTest {
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new LSMBTreeOperatorTestHelper(TestStorageManagerComponentHolder.getIOManager());
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
new file mode 100644
index 0000000..c1b1cd8
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.btree;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.tests.am.btree.BTreeSecondaryIndexSearchOperatorTest;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+
+public class LSMBTreeSecondaryIndexSearchOperatorTest extends BTreeSecondaryIndexSearchOperatorTest {
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new LSMBTreeOperatorTestHelper(TestStorageManagerComponentHolder.getIOManager());
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+        return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
new file mode 100644
index 0000000..84b34b7
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.rtree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+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.SynchronousSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
+
+public class LSMRTreeOperatorTestHelper extends LSMTreeOperatorTestHelper {
+
+    private static final int MERGE_THRESHOLD = 3;
+
+    public LSMRTreeOperatorTestHelper(IOManager ioManager) {
+        super(ioManager);
+    }
+
+    public IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
+        return new LSMRTreeDataflowHelperFactory(valueProviderFactories, rtreePolicyType, btreeComparatorFactories,
+                new ConstantMergePolicyProvider(MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
+                SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory,
+                DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES);
+    }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexInsertOperatorTest.java
new file mode 100644
index 0000000..99eb6c3
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexInsertOperatorTest.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.rtree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+import edu.uci.ics.hyracks.tests.am.rtree.RTreeSecondaryIndexInsertOperatorTest;
+
+public class LSMRTreeSecondaryIndexInsertOperatorTest extends RTreeSecondaryIndexInsertOperatorTest {
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new LSMRTreeOperatorTestHelper(TestStorageManagerComponentHolder.getIOManager());
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
+        return ((LSMRTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
+                rtreePolicyType, btreeComparatorFactories, linearizerCmpFactory);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexSearchOperatorTest.java
new file mode 100644
index 0000000..e1e6b04
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexSearchOperatorTest.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.rtree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+import edu.uci.ics.hyracks.tests.am.rtree.RTreeSecondaryIndexSearchOperatorTest;
+
+public class LSMRTreeSecondaryIndexSearchOperatorTest extends RTreeSecondaryIndexSearchOperatorTest {
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new LSMRTreeOperatorTestHelper(TestStorageManagerComponentHolder.getIOManager());
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
+        return ((LSMRTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
+                rtreePolicyType, btreeComparatorFactories, linearizerCmpFactory);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
new file mode 100644
index 0000000..bb31dcd
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.rtree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+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.SynchronousSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeWithAntiMatterTuplesDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
+
+public class LSMRTreeWithAntiMatterTuplesOperatorTestHelper extends LSMTreeOperatorTestHelper {
+
+    private static final int MERGE_THRESHOLD = 3;
+
+    public LSMRTreeWithAntiMatterTuplesOperatorTestHelper(IOManager ioManager) {
+        super(ioManager);
+    }
+
+    public IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
+        return new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(valueProviderFactories, rtreePolicyType,
+                btreeComparatorFactories, new ConstantMergePolicyProvider(MERGE_THRESHOLD),
+                ThreadCountingOperationTrackerFactory.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
+                NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory);
+    }
+
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest.java
new file mode 100644
index 0000000..bbf4dd3
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.rtree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+import edu.uci.ics.hyracks.tests.am.rtree.RTreeSecondaryIndexInsertOperatorTest;
+
+public class LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest extends RTreeSecondaryIndexInsertOperatorTest {
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new LSMRTreeWithAntiMatterTuplesOperatorTestHelper(TestStorageManagerComponentHolder.getIOManager());
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
+        return ((LSMRTreeWithAntiMatterTuplesOperatorTestHelper) testHelper).createDataFlowHelperFactory(
+                secondaryValueProviderFactories, rtreePolicyType, btreeComparatorFactories, linearizerCmpFactory);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
new file mode 100644
index 0000000..b615636
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2010 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.tests.am.lsm.rtree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+import edu.uci.ics.hyracks.tests.am.rtree.RTreeSecondaryIndexSearchOperatorTest;
+
+public class LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest extends RTreeSecondaryIndexSearchOperatorTest {
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new LSMRTreeWithAntiMatterTuplesOperatorTestHelper(TestStorageManagerComponentHolder.getIOManager());
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
+        return ((LSMRTreeWithAntiMatterTuplesOperatorTestHelper) testHelper).createDataFlowHelperFactory(
+                secondaryValueProviderFactories, rtreePolicyType, btreeComparatorFactories, linearizerCmpFactory);
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..c4a63df
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -0,0 +1,369 @@
+/*
+ * Copyright 2009-2010 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.tests.am.rtree;
+
+import java.io.DataOutput;
+import java.io.File;
+
+import org.junit.After;
+import org.junit.Before;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+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.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
+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.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+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.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.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.test.support.TestIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
+import edu.uci.ics.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+
+public abstract class AbstractRTreeOperatorTest extends AbstractIntegrationTest {
+    static {
+        TestStorageManagerComponentHolder.init(8192, 20, 20);
+    }
+
+    protected final IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+    protected final IIndexLifecycleManagerProvider lcManagerProvider = new TestIndexLifecycleManagerProvider();
+    protected IIndexDataflowHelperFactory rtreeDataflowHelperFactory;
+    protected IIndexDataflowHelperFactory btreeDataflowHelperFactory = new BTreeDataflowHelperFactory();
+
+    // field, type and key declarations for primary index
+    protected final int primaryFieldCount = 10;
+    protected final ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
+    protected final int primaryKeyFieldCount = 1;
+    protected final IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+
+    protected final RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE });
+
+    // to be set by subclasses
+    protected String primaryFileName;
+    protected IFileSplitProvider primarySplitProvider;
+
+    // field, type and key declarations for secondary indexes
+    protected final int secondaryFieldCount = 5;
+    protected final ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
+    protected final int secondaryKeyFieldCount = 4;
+    protected final IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
+
+    protected final RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE });
+
+    // This is only used for the LSMRTree. We need a comparator Factories for
+    // the BTree component of the LSMRTree.
+    protected final int btreeKeyFieldCount = 5;
+    protected final IBinaryComparatorFactory[] btreeComparatorFactories = new IBinaryComparatorFactory[btreeKeyFieldCount];
+
+    protected String secondaryFileName;
+    protected IFileSplitProvider secondarySplitProvider;
+
+    protected ITreeIndexOperatorTestHelper testHelper;
+
+    protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksException {
+        return new RTreeOperatorTestHelper();
+    }
+
+    @Before
+    public void setup() throws Exception {
+        testHelper = createTestHelper();
+
+        primaryFileName = testHelper.getPrimaryIndexName();
+        primarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID, new FileReference(
+                new File(primaryFileName))) });
+        secondaryFileName = testHelper.getSecondaryIndexName();
+        secondarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                new FileReference(new File(secondaryFileName))) });
+
+        // field, type and key declarations for primary index
+        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[6] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[7] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[8] = UTF8StringPointable.TYPE_TRAITS;
+        primaryTypeTraits[9] = UTF8StringPointable.TYPE_TRAITS;
+        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
+        // field, type and key declarations for secondary indexes
+        secondaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
+        secondaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
+        secondaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
+        secondaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
+        secondaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
+        secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
+        secondaryComparatorFactories[2] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
+        secondaryComparatorFactories[3] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
+
+        // This only used for LSMRTree
+        btreeComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
+        btreeComparatorFactories[1] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
+        btreeComparatorFactories[2] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
+        btreeComparatorFactories[3] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
+        btreeComparatorFactories[4] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
+        IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = RTreeUtils
+                .createPrimitiveValueProviderFactories(secondaryComparatorFactories.length, DoublePointable.FACTORY);
+
+        rtreeDataflowHelperFactory = createDataFlowHelperFactory(secondaryValueProviderFactories,
+                RTreePolicyType.RSTARTREE, btreeComparatorFactories,
+                LSMRTreeUtils.proposeBestLinearizer(secondaryTypeTraits, secondaryComparatorFactories.length));
+
+    }
+
+    protected abstract IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory)
+            throws TreeIndexException;
+
+    protected void createPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
+                lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories, null,
+                btreeDataflowHelperFactory, localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
+        spec.addRoot(primaryCreateOp);
+        runTest(spec);
+    }
+
+    protected void loadPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/orders-with-locations-part1.txt"))) };
+        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE });
+
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE,
+                        DoubleParserFactory.INSTANCE }, '|'), ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
+
+        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
+
+        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,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBulkLoad, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBulkLoad, 0);
+
+        spec.addRoot(primaryBulkLoad);
+        runTest(spec);
+    }
+
+    protected void createSecondaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+        TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+                storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, null, rtreeDataflowHelperFactory, localResourceFactoryProvider,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryCreateOp, NC1_ID);
+        spec.addRoot(secondaryCreateOp);
+        runTest(spec);
+    }
+
+    protected void loadSecondaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build dummy tuple containing nothing
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] lowKeyFields = null; // - infinity
+        int[] highKeyFields = null; // + infinity
+
+        // scan primary index
+        BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                null, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
+
+        // load secondary index
+        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);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBulkLoad, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primarySearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primarySearchOp, 0, secondaryBulkLoad, 0);
+
+        spec.addRoot(secondaryBulkLoad);
+        runTest(spec);
+    }
+
+    protected void insertPipeline() throws Exception {
+
+        JobSpecification spec = new JobSpecification();
+
+        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/orders-with-locations-part2.txt"))) };
+        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE });
+
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE,
+                        DoubleParserFactory.INSTANCE }, '|'), ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
+
+        // insert into primary index
+        int[] primaryFieldPermutation = { 0, 1, 2, 4, 5, 7, 9, 10, 11, 12 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor primaryInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, ordersDesc, storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits,
+                primaryComparatorFactories, null, primaryFieldPermutation, IndexOperation.INSERT,
+                btreeDataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryInsertOp, NC1_ID);
+
+        // secondary index
+        int[] secondaryFieldPermutation = { 9, 10, 11, 12, 0 };
+        TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+                spec, ordersDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, null, secondaryFieldPermutation, IndexOperation.INSERT,
+                rtreeDataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryInsertOp, NC1_ID);
+
+        NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, nullSink, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, primaryInsertOp, 0);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryInsertOp, 0, secondaryInsertOp, 0);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryInsertOp, 0, nullSink, 0);
+
+        spec.addRoot(nullSink);
+        runTest(spec);
+    }
+
+    protected void destroyPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IndexDropOperatorDescriptor primaryDropOp = new IndexDropOperatorDescriptor(spec, storageManager,
+                lcManagerProvider, primarySplitProvider, btreeDataflowHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryDropOp, NC1_ID);
+        spec.addRoot(primaryDropOp);
+        runTest(spec);
+    }
+
+    protected void destroySecondaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IndexDropOperatorDescriptor secondaryDropOp = new IndexDropOperatorDescriptor(spec, storageManager,
+                lcManagerProvider, secondarySplitProvider, rtreeDataflowHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryDropOp, NC1_ID);
+        spec.addRoot(secondaryDropOp);
+        runTest(spec);
+    }
+
+    @After
+    public abstract void cleanup() throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
new file mode 100644
index 0000000..85abcb1
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeOperatorTestHelper.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 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.tests.am.rtree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.tests.am.common.TreeOperatorTestHelper;
+
+public class RTreeOperatorTestHelper extends TreeOperatorTestHelper {
+
+    public IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories) {
+        return new RTreeDataflowHelperFactory(valueProviderFactories, rtreePolicyType);
+    }
+
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
new file mode 100644
index 0000000..8e68bcf
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2009-2010 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.tests.am.rtree;
+
+import java.io.DataOutput;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class RTreeSecondaryIndexInsertOperatorTest extends AbstractRTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+        createSecondaryIndex();
+        loadSecondaryIndex();
+        insertPipeline();
+    }
+
+    @Test
+    public void searchUpdatedSecondaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build tuple
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(61.2894, dos);
+        tb.addFieldEndOffset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(-149.624, dos);
+        tb.addFieldEndOffset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(61.8894, dos);
+        tb.addFieldEndOffset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(-149.024, dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] keyFields = { 0, 1, 2, 3 };
+
+        RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
+                storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
+
+        // fifth field from the tuples coming from secondary index
+        int[] primaryLowKeyFields = { 4 };
+        // fifth field from the tuples coming from secondary index
+        int[] primaryHighKeyFields = { 4 };
+
+        // search primary index
+        BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                null, primaryLowKeyFields, primaryHighKeyFields, true, true, btreeDataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondarySearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondarySearchOp, 0, primarySearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primarySearchOp, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory)
+            throws TreeIndexException {
+        return ((RTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
+                rtreePolicyType, null);
+    }
+
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+        destroySecondaryIndex();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
new file mode 100644
index 0000000..90bb918
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2009-2010 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.tests.am.rtree;
+
+import java.io.DataOutput;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class RTreeSecondaryIndexScanOperatorTest extends AbstractRTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+        createSecondaryIndex();
+        loadSecondaryIndex();
+    }
+
+    @Test
+    public void scanPrimaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build dummy tuple
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(0.0, dos);
+        tb.addFieldEndOffset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(0.0, dos);
+        tb.addFieldEndOffset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(0.0, dos);
+        tb.addFieldEndOffset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(0.0, dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] keyFields = null;
+
+        RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
+                storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondarySearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondarySearchOp, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
+        return ((RTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
+                rtreePolicyType, null);
+    }
+    
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+        destroySecondaryIndex();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
new file mode 100644
index 0000000..5da122c
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2009-2010 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.tests.am.rtree;
+
+import java.io.DataOutput;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class RTreeSecondaryIndexSearchOperatorTest extends AbstractRTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+        createSecondaryIndex();
+        loadSecondaryIndex();
+    }
+
+    @Test
+    public void searchSecondaryIndexTest() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        // build tuple
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+        DataOutput dos = tb.getDataOutput();
+
+        tb.reset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(61.2894, dos);
+        tb.addFieldEndOffset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(-149.624, dos);
+        tb.addFieldEndOffset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(61.8894, dos);
+        tb.addFieldEndOffset();
+        DoubleSerializerDeserializer.INSTANCE.serialize(-149.024, dos);
+        tb.addFieldEndOffset();
+
+        ISerializerDeserializer[] keyRecDescSers = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+        int[] keyFields = { 0, 1, 2, 3 };
+
+        RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
+                storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
+                secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
+
+        // fifth field from the tuples coming from secondary index
+        int[] primaryLowKeyFields = { 4 };
+        // fifth field from the tuples coming from secondary index
+        int[] primaryHighKeyFields = { 4 };
+
+        // search primary index
+        BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+                null, primaryLowKeyFields, primaryHighKeyFields, true, true, btreeDataflowHelperFactory, false,
+                NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
+
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondarySearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondarySearchOp, 0, primarySearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primarySearchOp, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory)
+            throws TreeIndexException {
+        return ((RTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
+                rtreePolicyType, null);
+    }
+
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+        destroySecondaryIndex();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexStatsOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexStatsOperatorTest.java
new file mode 100644
index 0000000..83be1d9
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexStatsOperatorTest.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 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.tests.am.rtree;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexStatsOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class RTreeSecondaryIndexStatsOperatorTest extends AbstractRTreeOperatorTest {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        createPrimaryIndex();
+        loadPrimaryIndex();
+        createSecondaryIndex();
+        loadSecondaryIndex();
+    }
+
+    @Test
+    public void showPrimaryIndexStats() throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        TreeIndexStatsOperatorDescriptor secondaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
+                lcManagerProvider, secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories, null,
+                rtreeDataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryStatsOp, NC1_ID);
+        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+                createTempFile().getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryStatsOp, 0, printer, 0);
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    @Override
+    protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
+            IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
+            IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
+        return ((RTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
+                rtreePolicyType, null);
+    }
+
+    @Override
+    public void cleanup() throws Exception {
+        destroyPrimaryIndex();
+        destroySecondaryIndex();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
deleted file mode 100644
index 33ddca2..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * Copyright 2009-2010 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.tests.btree;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-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.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class BTreePrimaryIndexScanOperatorTest extends AbstractIntegrationTest {
-    static {
-        TestStorageManagerComponentHolder.init(8192, 20, 20);
-    }
-
-    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-    private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
-
-    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
-    private final static String sep = System.getProperty("file.separator");
-
-    // field, type and key declarations for primary index
-    private int primaryFieldCount = 6;
-    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
-    private int primaryKeyFieldCount = 1;
-    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-
-    private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
-    private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
-
-    private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-
-    private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-    @Before
-    public void setup() throws Exception {
-        // field, type and key declarations for primary index
-        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
-        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        createPrimaryIndex();
-        loadPrimaryIndexTest();
-    }
-
-    public void createPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-
-    public void loadPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/tpch0.001/orders-part1.tbl"))) };
-        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
-
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
-
-        int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
-        TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
-
-        spec.addRoot(primaryBtreeBulkLoad);
-        runTest(spec);
-    }
-
-    @Test
-    public void scanPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        // build dummy tuple containing nothing
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-
-        int[] lowKeyFields = null; // - infinity
-        int[] highKeyFields = null; // + infinity
-
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @AfterClass
-    public static void cleanup() throws Exception {
-        File primary = new File(primaryFileName);
-        primary.deleteOnExit();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
deleted file mode 100644
index acd3027..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Copyright 2009-2010 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.tests.btree;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-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.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class BTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
-    static {
-        TestStorageManagerComponentHolder.init(8192, 20, 20);
-    }
-
-    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-    private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
-
-    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
-    private final static String sep = System.getProperty("file.separator");
-
-    // field, type and key declarations for primary index
-    private int primaryFieldCount = 6;
-    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
-    private int primaryKeyFieldCount = 1;
-    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-
-    private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
-    private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
-
-    private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-
-    private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-    @Before
-    public void setup() throws Exception {
-        // field, type and key declarations for primary index
-        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
-        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        createPrimaryIndex();
-        loadPrimaryIndexTest();
-    }
-
-    public void createPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-
-    public void loadPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/tpch0.001/orders-part1.tbl"))) };
-        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
-
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
-
-        int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
-        TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
-
-        spec.addRoot(primaryBtreeBulkLoad);
-        runTest(spec);
-    }
-
-    @Test
-    public void searchPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        // build tuple containing low and high search key
-        // high key and low key
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        // low key
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("100", dos);
-        tb.addFieldEndOffset();
-        // high key
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("200", dos);
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-
-        int[] lowKeyFields = { 0 };
-        int[] highKeyFields = { 1 };
-
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @AfterClass
-    public static void cleanup() throws Exception {
-        File primary = new File(primaryFileName);
-        primary.deleteOnExit();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
deleted file mode 100644
index ca03b16..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * Copyright 2009-2010 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.tests.btree;
-
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-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.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-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.dataflow.TreeIndexStatsOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class BTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
-    static {
-        TestStorageManagerComponentHolder.init(8192, 20, 20);
-    }
-
-    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-    private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
-
-    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
-    private final static String sep = System.getProperty("file.separator");
-
-    // field, type and key declarations for primary index
-    private int primaryFieldCount = 6;
-    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
-    private int primaryKeyFieldCount = 1;
-    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-
-    private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
-    private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
-
-    private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-
-    @Before
-    public void setup() throws Exception {
-        // field, type and key declarations for primary index
-        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
-        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        createPrimaryIndex();
-        loadPrimaryIndexTest();
-    }
-
-    public void createPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-
-    public void loadPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/tpch0.001/orders-part1.tbl"))) };
-        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
-
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
-
-        int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
-        TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
-
-        spec.addRoot(primaryBtreeBulkLoad);
-        runTest(spec);
-    }
-
-    @Test
-    public void showPrimaryIndexStats() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @AfterClass
-    public static void cleanup() throws Exception {
-        File primary = new File(primaryFileName);
-        primary.deleteOnExit();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
deleted file mode 100644
index 34a1cd3..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ /dev/null
@@ -1,367 +0,0 @@
-/*
- * Copyright 2009-2010 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.tests.btree;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-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.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-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.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class BTreeSecondaryIndexInsertOperatorTest extends AbstractIntegrationTest {
-    static {
-        TestStorageManagerComponentHolder.init(8192, 20, 20);
-    }
-
-    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-    private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
-
-    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
-    private final static String sep = System.getProperty("file.separator");
-
-    // field, type and key declarations for primary index
-    private int primaryFieldCount = 6;
-    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
-    private int primaryKeyFieldCount = 1;
-    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-
-    private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
-    private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
-
-    private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-
-    private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-    // field, type and key declarations for secondary indexes
-    private int secondaryFieldCount = 2;
-    private ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
-    private int secondaryKeyFieldCount = 2;
-    private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
-
-    private static String secondaryBtreeName = "secondary" + simpleDateFormat.format(new Date());
-    private static String secondaryFileName = System.getProperty("java.io.tmpdir") + sep + secondaryBtreeName;
-
-    private IFileSplitProvider secondaryBtreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(secondaryFileName))) });
-
-    private RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-    @Before
-    public void setup() throws Exception {
-        // field, type and key declarations for primary index
-        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
-        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        // field, type and key declarations for secondary indexes
-        secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
-        secondaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
-        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-        secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        createPrimaryIndex();
-        loadPrimaryIndexTest();
-        createSecondaryIndex();
-        loadSecondaryIndexTest();
-        insertPipelineTest();
-    }
-
-    public void createPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-    
-    public void loadPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/tpch0.001/orders-part1.tbl"))) };
-        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
-
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
-
-        int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
-        TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
-
-        spec.addRoot(primaryBtreeBulkLoad);
-        runTest(spec);
-    }
-
-    public void createSecondaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryCreateOp, NC1_ID);
-        spec.addRoot(secondaryCreateOp);
-        runTest(spec);
-    }
-    
-    public void loadSecondaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        // build dummy tuple containing nothing
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-
-        int[] lowKeyFields = null; // - infinity
-        int[] highKeyFields = null; // + infinity
-
-        // scan primary index
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
-                highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
-
-        // sort based on secondary keys
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 3, 0 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                primaryRecDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
-
-        // load secondary index
-        int[] fieldPermutation = { 3, 0 };
-        TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, sorter, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, secondaryBtreeBulkLoad, 0);
-
-        spec.addRoot(secondaryBtreeBulkLoad);
-        runTest(spec);
-    }
-
-    public void insertPipelineTest() throws Exception {
-
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/tpch0.001/orders-part2.tbl"))) };
-        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
-
-        // insert into primary index
-        int[] primaryFieldPermutation = { 0, 1, 2, 4, 5, 7 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor primaryBtreeInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, ordersDesc, storageManager, indexRegistryProvider, primaryBtreeSplitProvider,
-                primaryTypeTraits, primaryComparatorFactories,
-                primaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory, null, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeInsertOp, NC1_ID);
-
-        // first secondary index
-        int[] fieldPermutationB = { 4, 0 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, ordersDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
-                secondaryTypeTraits,
-                secondaryComparatorFactories, fieldPermutationB, IndexOp.INSERT, dataflowHelperFactory, null, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryInsertOp, NC1_ID);
-
-        NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, nullSink, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, primaryBtreeInsertOp, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeInsertOp, 0, secondaryInsertOp, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryInsertOp, 0, nullSink, 0);
-
-        spec.addRoot(nullSink);
-        runTest(spec);
-    }
-
-    @Test
-    public void searchUpdatedSecondaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        // build tuple containing search keys (only use the first key as search
-        // key)
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        // low key
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
-        tb.addFieldEndOffset();
-        // high key
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-
-        int[] secondaryLowKeyFields = { 0 };
-        int[] secondaryHighKeyFields = { 1 };
-
-        // search secondary index
-        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                secondaryRecDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
-                secondaryTypeTraits,
-                secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
-                dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
-
-        // second field from the tuples coming from secondary index
-        int[] primaryLowKeyFields = { 1 };
-        // second field from the tuples coming from secondary index
-        int[] primaryHighKeyFields = { 1 };
-
-        // search primary index
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, primaryLowKeyFields,
-                primaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @AfterClass
-    public static void cleanup() throws Exception {
-        File primary = new File(primaryFileName);
-        primary.deleteOnExit();
-
-        File secondary = new File(secondaryFileName);
-        secondary.deleteOnExit();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
deleted file mode 100644
index c9ee118..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/*
- * Copyright 2009-2010 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.tests.btree;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-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.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class BTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
-    static {
-        TestStorageManagerComponentHolder.init(8192, 20, 20);
-    }
-
-    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-    private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
-
-    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
-    private final static String sep = System.getProperty("file.separator");
-
-    // field, type and key declarations for primary index
-    private int primaryFieldCount = 6;
-    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
-    private int primaryKeyFieldCount = 1;
-    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-
-    private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
-    private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
-
-    private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-
-    private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-    // field, type and key declarations for secondary indexes
-    private int secondaryFieldCount = 2;
-    private ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
-    private int secondaryKeyFieldCount = 2;
-    private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
-
-    private static String secondaryBtreeName = "secondary" + simpleDateFormat.format(new Date());
-    private static String secondaryFileName = System.getProperty("java.io.tmpdir") + sep + secondaryBtreeName;
-
-    private IFileSplitProvider secondaryBtreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(secondaryFileName))) });
-
-    private RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-    @Before
-    public void setup() throws Exception {
-        // field, type and key declarations for primary index
-        primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
-        primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
-        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        // field, type and key declarations for secondary indexes
-        secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
-        secondaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
-        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-        secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        createPrimaryIndex();
-        loadPrimaryIndexTest();
-        createSecondaryIndex();
-        loadSecondaryIndexTest();
-    }
-
-    public void createPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-
-    public void loadPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/tpch0.001/orders-part1.tbl"))) };
-        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
-
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
-
-        int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
-        TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
-
-        spec.addRoot(primaryBtreeBulkLoad);
-        runTest(spec);
-    }
-
-    public void createSecondaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits,
-                secondaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryCreateOp, NC1_ID);
-        spec.addRoot(secondaryCreateOp);
-        runTest(spec);
-    }
-
-    public void loadSecondaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        // build dummy tuple containing nothing
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-
-        int[] lowKeyFields = null; // - infinity
-        int[] highKeyFields = null; // + infinity
-
-        // scan primary index
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
-
-        // sort based on secondary keys
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 3, 0 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                primaryRecDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
-
-        // load secondary index
-        int[] fieldPermutation = { 3, 0 };
-        TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits,
-                secondaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, sorter, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, secondaryBtreeBulkLoad, 0);
-
-        spec.addRoot(secondaryBtreeBulkLoad);
-        runTest(spec);
-    }
-
-    @Test
-    public void searchSecondaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        // build tuple containing search keys (only use the first key as search
-        // key)
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        // low key
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
-        tb.addFieldEndOffset();
-        // high key
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-
-        int[] secondaryLowKeyFields = { 0 };
-        int[] secondaryHighKeyFields = { 1 };
-
-        // search secondary index
-        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                secondaryRecDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
-                secondaryTypeTraits, secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true,
-                true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
-
-        int[] primaryLowKeyFields = { 1 }; // second field from the tuples
-        // coming from secondary index
-        int[] primaryHighKeyFields = { 1 }; // second field from the tuples
-        // coming from secondary index
-
-        // search primary index
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
-                dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @AfterClass
-    public static void cleanup() throws Exception {
-        File primary = new File(primaryFileName);
-        primary.deleteOnExit();
-        File secondary = new File(secondaryFileName);
-        secondary.deleteOnExit();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index acaa94a..e74e54c 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -60,6 +60,9 @@
     private static IHyracksClientConnection hcc;
 
     private final List<File> outputFiles;
+    
+    protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
+    protected static int DEFAULT_MEM_NUM_PAGES = 1000;
 
     @Rule
     public TemporaryFolder outputFolder = new TemporaryFolder();
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
deleted file mode 100644
index 600b54b..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
+++ /dev/null
@@ -1,77 +0,0 @@
-package edu.uci.ics.hyracks.tests.invertedindex;
-
-import java.io.File;
-
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class BinaryTokenizerOperatorTest extends AbstractIntegrationTest {
-
-    @Test
-    public void tokenizerTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] dblpTitleFileSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/cleanednumbereddblptitles.txt"))) };
-        IFileSplitProvider dblpTitleSplitProvider = new ConstantFileSplitProvider(dblpTitleFileSplits);
-        RecordDescriptor dblpTitleRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dblpTitleScanner, NC1_ID);
-
-        RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-
-        ITokenFactory tokenFactory = new UTF8WordTokenFactory();
-        IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
-                tokenFactory);
-        int[] tokenFields = { 1 };
-        int[] keyFields = { 0 };
-        BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
-                tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), binaryTokenizer, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
deleted file mode 100644
index b5a4df8..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
+++ /dev/null
@@ -1,77 +0,0 @@
-package edu.uci.ics.hyracks.tests.invertedindex;
-
-import java.io.File;
-
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class InvertedIndexOperatorsTest extends AbstractIntegrationTest {
-
-    @Test
-    public void tokenizerTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] dblpTitleFileSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/cleanednumbereddblptitles.txt"))) };
-        IFileSplitProvider dblpTitleSplitProvider = new ConstantFileSplitProvider(dblpTitleFileSplits);
-        RecordDescriptor dblpTitleRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dblpTitleScanner, NC1_ID);
-
-        RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-
-        ITokenFactory tokenFactory = new UTF8WordTokenFactory();
-        IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
-                tokenFactory);
-        int[] tokenFields = { 1 };
-        int[] projFields = { 0 };
-        BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
-                tokenizerRecDesc, tokenizerFactory, tokenFields, projFields);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), binaryTokenizer, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
deleted file mode 100644
index d1071a3..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
+++ /dev/null
@@ -1,358 +0,0 @@
-/*
- * Copyright 2009-2010 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.tests.invertedindex;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-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.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexBulkLoadOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexCreateOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.ConjunctiveSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class WordInvertedIndexTest extends AbstractIntegrationTest {
-    static {
-        TestStorageManagerComponentHolder.init(8192, 20, 20);
-    }
-
-    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-    private IIndexDataflowHelperFactory btreeDataflowHelperFactory = new BTreeDataflowHelperFactory();
-
-    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
-    private final static String sep = System.getProperty("file.separator");
-    private final static String dateString = simpleDateFormat.format(new Date());
-    private final static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree"
-            + dateString;
-    private final static String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree"
-            + dateString;
-    private final static String invListsFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexLists"
-            + dateString;
-
-    private IFileSplitProvider primaryFileSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-    private IFileSplitProvider btreeFileSplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(
-            NC1_ID, new FileReference(new File(btreeFileName))) });
-    private IFileSplitProvider invListsFileSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(invListsFileName))) });
-
-    // Primary BTree index.
-    private int primaryFieldCount = 2;
-    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
-    private int primaryKeyFieldCount = 1;
-    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-    private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-    // Inverted index BTree dictionary.
-    private ITypeTraits[] tokenTypeTraits = new ITypeTraits[1];
-    private IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[1];
-
-    // Inverted index stuff.
-    private int invListElementFieldCount = 1;
-    private ITypeTraits[] invListsTypeTraits = new ITypeTraits[invListElementFieldCount];
-    private IBinaryComparatorFactory[] invListsComparatorFactories = new IBinaryComparatorFactory[invListElementFieldCount];
-    private RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-    private RecordDescriptor invListsRecDesc = new RecordDescriptor(
-            new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
-
-    // Tokenizer stuff.
-    private ITokenFactory tokenFactory = new UTF8WordTokenFactory();
-    private IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
-            tokenFactory);
-
-    @Before
-    public void setup() throws Exception {
-        // Field declarations and comparators for primary BTree index.
-        primaryTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
-        primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
-        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-
-        // Field declarations and comparators for tokens.
-        tokenTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
-        tokenComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        // Field declarations and comparators for inverted lists.
-        invListsTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
-        invListsComparatorFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-
-        createPrimaryIndex();
-        loadPrimaryIndex();
-        printPrimaryIndex();
-        createInvertedIndex();
-        loadInvertedIndex();
-    }
-
-    public void createPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-
-    @Test
-    public void testConjunctiveSearcher() throws Exception {
-        IInvertedIndexSearchModifierFactory conjunctiveSearchModifierFactory = new ConjunctiveSearchModifierFactory();
-        searchInvertedIndex("of", conjunctiveSearchModifierFactory);
-        searchInvertedIndex("3d", conjunctiveSearchModifierFactory);
-        searchInvertedIndex("of the human", conjunctiveSearchModifierFactory);
-    }
-
-    private IOperatorDescriptor createFileScanOp(JobSpecification spec) {
-        FileSplit[] dblpTitleFileSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/cleanednumbereddblptitles.txt"))) };
-        IFileSplitProvider dblpTitleSplitProvider = new ConstantFileSplitProvider(dblpTitleFileSplits);
-        RecordDescriptor dblpTitleRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-        FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dblpTitleScanner, NC1_ID);
-        return dblpTitleScanner;
-    }
-
-    private IOperatorDescriptor createPrimaryBulkLoadOp(JobSpecification spec) {
-        int[] fieldPermutation = { 0, 1 };
-        TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, fieldPermutation, 0.7f, btreeDataflowHelperFactory,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
-        return primaryBtreeBulkLoad;
-    }
-
-    private IOperatorDescriptor createScanKeyProviderOp(JobSpecification spec) throws HyracksDataException {
-        // build dummy tuple containing nothing
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
-        DataOutput dos = tb.getDataOutput();
-        tb.reset();
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
-        tb.addFieldEndOffset();
-        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-        return keyProviderOp;
-    }
-
-    private IOperatorDescriptor createPrimaryScanOp(JobSpecification spec) throws HyracksDataException {
-        int[] lowKeyFields = null; // - infinity
-        int[] highKeyFields = null; // + infinity
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits,
-                primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory, false,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
-        return primaryBtreeSearchOp;
-    }
-
-    private void loadPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        // Assuming that the data is pre-sorted on the key. No need to sort
-        // before bulk load.
-        IOperatorDescriptor fileScanOp = createFileScanOp(spec);
-        IOperatorDescriptor primaryBulkLoad = createPrimaryBulkLoadOp(spec);
-        spec.connect(new OneToOneConnectorDescriptor(spec), fileScanOp, 0, primaryBulkLoad, 0);
-        spec.addRoot(primaryBulkLoad);
-        runTest(spec);
-    }
-
-    private void printPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
-        IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, printer, 0);
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    private IOperatorDescriptor createExternalSortOp(JobSpecification spec, int[] sortFields,
-            RecordDescriptor outputRecDesc) {
-        ExternalSortOperatorDescriptor externalSortOp = new ExternalSortOperatorDescriptor(spec, 1000, sortFields,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, outputRecDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, externalSortOp, NC1_ID);
-        return externalSortOp;
-    }
-
-    private IOperatorDescriptor createBinaryTokenizerOp(JobSpecification spec, int[] tokenFields, int[] keyFields) {
-        BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
-                tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
-        return binaryTokenizer;
-    }
-
-    private IOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec, int[] fieldPermutation) {
-        InvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new InvertedIndexBulkLoadOperatorDescriptor(spec,
-                fieldPermutation, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
-                indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
-                invListsComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexBulkLoadOp, NC1_ID);
-        return invIndexBulkLoadOp;
-    }
-
-    public void createInvertedIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        InvertedIndexCreateOperatorDescriptor invIndexCreateOp = new InvertedIndexCreateOperatorDescriptor(spec,
-                storageManager, btreeFileSplitProvider, invListsFileSplitProvider, indexRegistryProvider,
-                tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
-                tokenizerFactory, btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexCreateOp, NC1_ID);
-        spec.addRoot(invIndexCreateOp);
-        runTest(spec);
-    }
-
-    public void loadInvertedIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
-        IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
-        int[] tokenFields = { 1 };
-        int[] keyFields = { 0 };
-        IOperatorDescriptor binaryTokenizerOp = createBinaryTokenizerOp(spec, tokenFields, keyFields);
-        int[] sortFields = { 0, 1 };
-        IOperatorDescriptor externalSortOp = createExternalSortOp(spec, sortFields, tokenizerRecDesc);
-        int[] fieldPermutation = { 0, 1 };
-        IOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec, fieldPermutation);
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, binaryTokenizerOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), binaryTokenizerOp, 0, externalSortOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), externalSortOp, 0, invIndexBulkLoadOp, 0);
-        spec.addRoot(invIndexBulkLoadOp);
-        runTest(spec);
-    }
-
-    private IOperatorDescriptor createQueryProviderOp(JobSpecification spec, String queryString)
-            throws HyracksDataException {
-        // Build tuple with exactly one field, which is the query,
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
-        DataOutput dos = tb.getDataOutput();
-        tb.reset();
-        UTF8StringSerializerDeserializer.INSTANCE.serialize(queryString, dos);
-        tb.addFieldEndOffset();
-        ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
-        ConstantTupleSourceOperatorDescriptor queryProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                queryRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, queryProviderOp, NC1_ID);
-        return queryProviderOp;
-    }
-
-    private IOperatorDescriptor createInvertedIndexSearchOp(JobSpecification spec,
-            IInvertedIndexSearchModifierFactory searchModifierFactory) {
-        InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(spec, 0,
-                storageManager, btreeFileSplitProvider, invListsFileSplitProvider, indexRegistryProvider,
-                tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
-                btreeDataflowHelperFactory, tokenizerFactory, searchModifierFactory, invListsRecDesc, false,
-                NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexSearchOp, NC1_ID);
-        return invIndexSearchOp;
-    }
-
-    public void searchInvertedIndex(String queryString, IInvertedIndexSearchModifierFactory searchModifierFactory)
-            throws Exception {
-        JobSpecification spec = new JobSpecification();
-        IOperatorDescriptor queryProviderOp = createQueryProviderOp(spec, queryString);
-        IOperatorDescriptor invIndexSearchOp = createInvertedIndexSearchOp(spec, searchModifierFactory);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        spec.connect(new OneToOneConnectorDescriptor(spec), queryProviderOp, 0, invIndexSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), invIndexSearchOp, 0, printer, 0);
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @AfterClass
-    public static void cleanup() throws Exception {
-        File primary = new File(primaryFileName);
-        File btree = new File(btreeFileName);
-        File invLists = new File(invListsFileName);
-        primary.deleteOnExit();
-        btree.deleteOnExit();
-        invLists.deleteOnExit();
-    }
-}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
deleted file mode 100644
index 92b6e14..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Copyright 2009-2010 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.tests.rtree;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class RTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
-    static {
-        TestStorageManagerComponentHolder.init(8192, 20, 20);
-    }
-
-    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-    private IIndexDataflowHelperFactory dataflowHelperFactory;
-
-    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
-    private final static String sep = System.getProperty("file.separator");
-
-    // field, type and key declarations for primary R-tree index
-    private int primaryFieldCount = 5;
-    private int primaryKeyFieldCount = 4;
-    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
-    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-
-    private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE });
-
-    private static String primaryRTreeName = "primary" + simpleDateFormat.format(new Date());
-    private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryRTreeName;
-
-    private IFileSplitProvider primaryRTreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-
-    private IPrimitiveValueProviderFactory[] primaryValueProviderFactories;
-    
-    @Before
-    public void setup() throws Exception {
-        // field, type and key declarations for primary R-tree index
-        primaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
-        primaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
-        primaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
-        primaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
-        primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
-        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
-        primaryComparatorFactories[1] = primaryComparatorFactories[0];
-        primaryComparatorFactories[2] = primaryComparatorFactories[0];
-        primaryComparatorFactories[3] = primaryComparatorFactories[0];
-
-        primaryValueProviderFactories = RTreeUtils
-                .createPrimitiveValueProviderFactories(primaryComparatorFactories.length, DoublePointable.FACTORY);
-        dataflowHelperFactory = new RTreeDataflowHelperFactory(primaryValueProviderFactories);
-        
-        createPrimaryIndex();
-        loadPrimaryIndexTest();
-    }
-
-    public void createPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryRTreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-    
-    public void loadPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] objectsSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/spatial.txt"))) };
-        IFileSplitProvider objectsSplitProvider = new ConstantFileSplitProvider(objectsSplits);
-        RecordDescriptor objectsDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor objScanner = new FileScanOperatorDescriptor(spec, objectsSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { DoubleParserFactory.INSTANCE,
-                        DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), objectsDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, objScanner, NC1_ID);
-
-        int[] fieldPermutation = { 0, 1, 2, 3, 4 };
-        TreeIndexBulkLoadOperatorDescriptor primaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryRTreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryRTreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), objScanner, 0, primaryRTreeBulkLoad, 0);
-
-        spec.addRoot(primaryRTreeBulkLoad);
-        runTest(spec);
-    }
-
-    @Test
-    public void searchPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        // build tuple
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount);
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        DoubleSerializerDeserializer.INSTANCE.serialize(61.2894, dos);
-        tb.addFieldEndOffset();
-        DoubleSerializerDeserializer.INSTANCE.serialize(-149.624, dos);
-        tb.addFieldEndOffset();
-        DoubleSerializerDeserializer.INSTANCE.serialize(61.8894, dos);
-        tb.addFieldEndOffset();
-        DoubleSerializerDeserializer.INSTANCE.serialize(-149.024, dos);
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { DoubleSerializerDeserializer.INSTANCE,
-                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-                DoubleSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-
-        int[] keyFields = { 0, 1, 2, 3 };
-
-        RTreeSearchOperatorDescriptor primaryRTreeSearchOp = new RTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                storageManager, indexRegistryProvider, primaryRTreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, keyFields,
-                dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryRTreeSearchOp, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryRTreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryRTreeSearchOp, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @AfterClass
-    public static void cleanup() throws Exception {
-        File primary = new File(primaryFileName);
-        primary.deleteOnExit();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
deleted file mode 100644
index 2a00394..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Copyright 2009-2010 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.tests.rtree;
-
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-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.dataflow.TreeIndexStatsOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class RTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
-    static {
-        TestStorageManagerComponentHolder.init(8192, 20, 20);
-    }
-
-    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-    private IIndexDataflowHelperFactory dataflowHelperFactory;
-
-    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
-    private final static String sep = System.getProperty("file.separator");
-
-    // field, type and key declarations for primary R-tree index
-    private int primaryFieldCount = 5;
-    private int primaryKeyFieldCount = 4;
-    private ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
-    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-
-    private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
-            primaryTypeTraits);
-
-    private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE });
-
-    private ITreeIndexFrameFactory primaryInteriorFrameFactory;
-    private ITreeIndexFrameFactory primaryLeafFrameFactory;
-
-    private static String primaryRTreeName = "primary" + simpleDateFormat.format(new Date());
-    private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryRTreeName;
-
-    private IFileSplitProvider primaryRTreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-
-    private IPrimitiveValueProviderFactory[] primaryValueProviderFactories;
-    
-    @Before
-    public void setup() throws Exception {
-        // field, type and key declarations for primary R-tree index
-        primaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
-        primaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
-        primaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
-        primaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
-        primaryTypeTraits[4] = DoublePointable.TYPE_TRAITS;
-        primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
-        primaryComparatorFactories[1] = primaryComparatorFactories[0];
-        primaryComparatorFactories[2] = primaryComparatorFactories[0];
-        primaryComparatorFactories[3] = primaryComparatorFactories[0];
-
-        primaryValueProviderFactories = RTreeUtils
-                .createPrimitiveValueProviderFactories(primaryComparatorFactories.length, DoublePointable.FACTORY);
-        dataflowHelperFactory = new RTreeDataflowHelperFactory(primaryValueProviderFactories);
-        
-        primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(primaryTupleWriterFactory,
-                primaryValueProviderFactories);
-        primaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(primaryTupleWriterFactory, primaryValueProviderFactories);
-
-        createPrimaryIndex();
-        loadPrimaryIndexTest();
-    }
-
-    public void createPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryRTreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-    
-    public void loadPrimaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] objectsSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/spatial.txt"))) };
-        IFileSplitProvider objectsSplitProvider = new ConstantFileSplitProvider(objectsSplits);
-        RecordDescriptor objectsDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor objScanner = new FileScanOperatorDescriptor(spec, objectsSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { DoubleParserFactory.INSTANCE,
-                        DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), objectsDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, objScanner, NC1_ID);
-
-        int[] fieldPermutation = { 0, 1, 2, 3, 4 };
-        TreeIndexBulkLoadOperatorDescriptor primaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryRTreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryRTreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), objScanner, 0, primaryRTreeBulkLoad, 0);
-
-        spec.addRoot(primaryRTreeBulkLoad);
-        runTest(spec);
-    }
-
-    @Test
-    public void showPrimaryIndexStats() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryRTreeSplitProvider, 
-                primaryTypeTraits, primaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @AfterClass
-    public static void cleanup() throws Exception {
-        File primary = new File(primaryFileName);
-        primary.deleteOnExit();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
deleted file mode 100644
index 1d86037..0000000
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ /dev/null
@@ -1,324 +0,0 @@
-/*
- * Copyright 2009-2010 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.tests.rtree;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-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.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataset.ResultSetId;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-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.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
-
-public class RTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
-    static {
-        TestStorageManagerComponentHolder.init(8192, 20, 20);
-    }
-
-    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-    private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
-    private IIndexDataflowHelperFactory dataflowHelperFactory;
-    private IIndexDataflowHelperFactory btreeDataflowHelperFactory = new BTreeDataflowHelperFactory();
-
-    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
-    private final static String sep = System.getProperty("file.separator");
-
-    // field, type and key declarations for primary B-tree index
-    private int primaryBTreeFieldCount = 10;
-    private ITypeTraits[] primaryBTreeTypeTraits = new ITypeTraits[primaryBTreeFieldCount];
-    private int primaryBTreeKeyFieldCount = 1;
-    private IBinaryComparatorFactory[] primaryBTreeComparatorFactories = new IBinaryComparatorFactory[primaryBTreeKeyFieldCount];
-
-    private static String primaryBTreeName = "primaryBTree" + simpleDateFormat.format(new Date());
-    private static String primaryBTreeFileName = System.getProperty("java.io.tmpdir") + sep + primaryBTreeName;
-
-    private IFileSplitProvider primaryBTreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryBTreeFileName))) });
-
-    private RecordDescriptor primaryBTreeRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE, });
-
-    // field, type and key declarations for secondary indexes
-    private int secondaryFieldCount = 5;
-    private ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
-    private int secondaryKeyFieldCount = 4;
-    private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
-
-    private static String secondaryRTreeName = "secondary" + simpleDateFormat.format(new Date());
-    private static String secondaryFileName = System.getProperty("java.io.tmpdir") + sep + secondaryRTreeName;
-
-    private IFileSplitProvider secondaryRTreeSplitProvider = new ConstantFileSplitProvider(
-            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(secondaryFileName))) });
-
-    private RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-            DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE });
-
-    private IPrimitiveValueProviderFactory[] secondaryValueProviderFactories;
-    
-    @Before
-    public void setup() throws Exception {
-        // field, type and key declarations for primary B-tree index
-        primaryBTreeTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
-        primaryBTreeTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
-        primaryBTreeTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
-        primaryBTreeTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
-        primaryBTreeTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
-        primaryBTreeTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
-        primaryBTreeTypeTraits[6] = DoublePointable.TYPE_TRAITS;
-        primaryBTreeTypeTraits[7] = DoublePointable.TYPE_TRAITS;
-        primaryBTreeTypeTraits[8] = DoublePointable.TYPE_TRAITS;
-        primaryBTreeTypeTraits[9] = DoublePointable.TYPE_TRAITS;
-        primaryBTreeComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        // field, type and key declarations for secondary indexes
-        secondaryTypeTraits[0] = DoublePointable.TYPE_TRAITS;
-        secondaryTypeTraits[1] = DoublePointable.TYPE_TRAITS;
-        secondaryTypeTraits[2] = DoublePointable.TYPE_TRAITS;
-        secondaryTypeTraits[3] = DoublePointable.TYPE_TRAITS;
-        secondaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
-        secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
-        secondaryComparatorFactories[1] = secondaryComparatorFactories[0];
-        secondaryComparatorFactories[2] = secondaryComparatorFactories[0];
-        secondaryComparatorFactories[3] = secondaryComparatorFactories[0];
-
-        secondaryValueProviderFactories = RTreeUtils
-                .createPrimitiveValueProviderFactories(secondaryComparatorFactories.length, DoublePointable.FACTORY);
-
-        dataflowHelperFactory = new RTreeDataflowHelperFactory(secondaryValueProviderFactories);
-        
-        createPrimaryIndex();
-        loadPrimaryBTreeIndexTest();
-        createSecondaryIndex();
-        loadSecondaryIndexTest();
-    }
-
-    public void createPrimaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, primaryBTreeSplitProvider, primaryBTreeTypeTraits,
-                primaryBTreeComparatorFactories, btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-    
-    public void loadPrimaryBTreeIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-                "data/orders-with-locations.txt"))) };
-        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-                DoubleSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE,
-                        DoubleParserFactory.INSTANCE }, '|'), ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
-
-        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                ordersDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
-
-        int[] fieldPermutation = { 0, 1, 2, 4, 5, 7, 9, 10, 11, 12 };
-        TreeIndexBulkLoadOperatorDescriptor primaryBTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, primaryBTreeSplitProvider, primaryBTreeTypeTraits, primaryBTreeComparatorFactories,
-                fieldPermutation, 0.7f, btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBTreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBTreeBulkLoad, 0);
-
-        spec.addRoot(primaryBTreeBulkLoad);
-        runTest(spec);
-    }
-
-    public void createSecondaryIndex() throws Exception {
-        JobSpecification spec = new JobSpecification();
-        TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, secondaryRTreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
-        spec.addRoot(primaryCreateOp);
-        runTest(spec);
-    }
-    
-    public void loadSecondaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        // build dummy tuple containing nothing
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryBTreeKeyFieldCount * 2);
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-
-        int[] lowKeyFields = null; // - infinity
-        int[] highKeyFields = null; // + infinity
-
-        // scan primary index
-        BTreeSearchOperatorDescriptor primaryBTreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                primaryBTreeRecDesc, storageManager, indexRegistryProvider, primaryBTreeSplitProvider,
-                primaryBTreeTypeTraits, primaryBTreeComparatorFactories, lowKeyFields, highKeyFields, 
-                true, true, btreeDataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBTreeSearchOp, NC1_ID);
-
-        // load secondary index
-        int[] fieldPermutation = { 6, 7, 8, 9, 0 };
-        TreeIndexBulkLoadOperatorDescriptor secondaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, secondaryRTreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutation, 0.7f,
-                dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryRTreeBulkLoad, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBTreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryBTreeSearchOp, 0, secondaryRTreeBulkLoad, 0);
-
-        spec.addRoot(secondaryRTreeBulkLoad);
-        runTest(spec);
-    }
-
-    @Test
-    public void searchSecondaryIndexTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        // build tuple
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        DoubleSerializerDeserializer.INSTANCE.serialize(61.2894, dos);
-        tb.addFieldEndOffset();
-        DoubleSerializerDeserializer.INSTANCE.serialize(-149.624, dos);
-        tb.addFieldEndOffset();
-        DoubleSerializerDeserializer.INSTANCE.serialize(61.8894, dos);
-        tb.addFieldEndOffset();
-        DoubleSerializerDeserializer.INSTANCE.serialize(-149.024, dos);
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { DoubleSerializerDeserializer.INSTANCE,
-                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
-                DoubleSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
-
-        int[] keyFields = { 0, 1, 2, 3 };
-
-        RTreeSearchOperatorDescriptor secondaryRTreeSearchOp = new RTreeSearchOperatorDescriptor(spec,
-                secondaryRecDesc, storageManager, indexRegistryProvider, secondaryRTreeSplitProvider,
-                secondaryTypeTraits, secondaryComparatorFactories, keyFields, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryRTreeSearchOp, NC1_ID);
-
-        ResultSetId rsId = new ResultSetId(1);
-        spec.addResultSetId(rsId);
-
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryRTreeSearchOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), secondaryRTreeSearchOp, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @AfterClass
-    public static void cleanup() throws Exception {
-        File primaryBTree = new File(primaryBTreeFileName);
-        primaryBTree.deleteOnExit();
-
-        File secondary = new File(secondaryFileName);
-        secondary.deleteOnExit();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/text-example/pom.xml b/hyracks/hyracks-examples/text-example/pom.xml
index 469fd0e..748371f 100644
--- a/hyracks/hyracks-examples/text-example/pom.xml
+++ b/hyracks/hyracks-examples/text-example/pom.xml
@@ -1,6 +1,5 @@
 <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.examples</groupId>
   <artifactId>text-example</artifactId>
   <packaging>pom</packaging>
   <name>text-example</name>
diff --git a/hyracks/hyracks-examples/text-example/textclient/pom.xml b/hyracks/hyracks-examples/text-example/textclient/pom.xml
index 4aace73..44493d8 100644
--- a/hyracks/hyracks-examples/text-example/textclient/pom.xml
+++ b/hyracks/hyracks-examples/text-example/textclient/pom.xml
@@ -1,11 +1,10 @@
 <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.examples.text</groupId>
   <artifactId>textclient</artifactId>
   <name>textclient</name>
 
   <parent>
-    <groupId>edu.uci.ics.hyracks.examples</groupId>
+    <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>text-example</artifactId>
     <version>0.2.3-SNAPSHOT</version>
   </parent>
@@ -18,7 +17,7 @@
   		<scope>compile</scope>
   	</dependency>
   	<dependency>
-  		<groupId>edu.uci.ics.hyracks.examples.text</groupId>
+  		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>texthelper</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
   		<type>jar</type>
diff --git a/hyracks/hyracks-examples/text-example/texthelper/pom.xml b/hyracks/hyracks-examples/text-example/texthelper/pom.xml
index bcb280c..499480f 100644
--- a/hyracks/hyracks-examples/text-example/texthelper/pom.xml
+++ b/hyracks/hyracks-examples/text-example/texthelper/pom.xml
@@ -1,11 +1,10 @@
 <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.examples.text</groupId>
   <artifactId>texthelper</artifactId>
   <name>texthelper</name>
 
   <parent>
-    <groupId>edu.uci.ics.hyracks.examples</groupId>
+    <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>text-example</artifactId>
     <version>0.2.3-SNAPSHOT</version>
   </parent>
diff --git a/hyracks/hyracks-examples/tpch-example/pom.xml b/hyracks/hyracks-examples/tpch-example/pom.xml
index 7d676dc..79053e8 100644
--- a/hyracks/hyracks-examples/tpch-example/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/pom.xml
@@ -1,6 +1,5 @@
 <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.examples</groupId>
   <artifactId>tpch-example</artifactId>
   <packaging>pom</packaging>
   <name>tpch-example</name>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index 4e0d9f0..7614050 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -1,10 +1,9 @@
 <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.examples.tpch</groupId>
   <artifactId>tpchclient</artifactId>
   <name>tpchclient</name>
   <parent>
-    <groupId>edu.uci.ics.hyracks.examples</groupId>
+    <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>tpch-example</artifactId>
     <version>0.2.3-SNAPSHOT</version>
   </parent>
diff --git a/hyracks/hyracks-hadoop-compat/pom.xml b/hyracks/hyracks-hadoop-compat/pom.xml
index 87aaaa7..fc8943a 100644
--- a/hyracks/hyracks-hadoop-compat/pom.xml
+++ b/hyracks/hyracks-hadoop-compat/pom.xml
@@ -1,8 +1,6 @@
 <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>hyracks-hadoop-compat</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-hadoop-compat</name>
 
   <parent>
diff --git a/hyracks/hyracks-storage-am-bloomfilter/pom.xml b/hyracks/hyracks-storage-am-bloomfilter/pom.xml
new file mode 100644
index 0000000..84c78d2
--- /dev/null
+++ b/hyracks/hyracks-storage-am-bloomfilter/pom.xml
@@ -0,0 +1,40 @@
+<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>
+  <artifactId>hyracks-storage-am-bloomfilter</artifactId>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-common</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>  	
+  	<dependency>
+  		<groupId>junit</groupId>
+  		<artifactId>junit</artifactId>
+  		<version>4.8.1</version>
+  		<type>jar</type>
+  		<scope>test</scope>
+  	</dependency>  	  		
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomCalculations.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomCalculations.java
new file mode 100644
index 0000000..9c9a7be
--- /dev/null
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomCalculations.java
@@ -0,0 +1,163 @@
+/*
+ * 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.bloomfilter.impls;
+
+/**
+ * This class has been taken from cassandra source code with minor modifications.
+ */
+
+/**
+ * The following calculations are taken from:
+ * http://www.cs.wisc.edu/~cao/papers/summary-cache/node8.html
+ * "Bloom Filters - the math"
+ * This class's static methods are meant to facilitate the use of the Bloom
+ * Filter class by helping to choose correct values of 'bits per element' and
+ * 'number of hash functions, k'.
+ */
+public class BloomCalculations {
+
+    private static final int minBuckets = 2;
+    private static final int minK = 1;
+
+    /**
+     * In the following table, the row 'i' shows false positive rates if i buckets
+     * per element are used. Column 'j' shows false positive rates if j hash
+     * functions are used. The first row is 'i=0', the first column is 'j=0'.
+     * Each cell (i,j) the false positive rate determined by using i buckets per
+     * element and j hash functions.
+     */
+    static final double[][] probs = new double[][] {
+            { 1.0 }, // dummy row representing 0 buckets per element
+            { 1.0, 1.0 }, // dummy row representing 1 buckets per element
+            { 1.0, 0.393, 0.400 },
+            { 1.0, 0.283, 0.237, 0.253 },
+            { 1.0, 0.221, 0.155, 0.147, 0.160 },
+            { 1.0, 0.181, 0.109, 0.092, 0.092, 0.101 }, // 5
+            { 1.0, 0.154, 0.0804, 0.0609, 0.0561, 0.0578, 0.0638 },
+            { 1.0, 0.133, 0.0618, 0.0423, 0.0359, 0.0347, 0.0364 },
+            { 1.0, 0.118, 0.0489, 0.0306, 0.024, 0.0217, 0.0216, 0.0229 },
+            { 1.0, 0.105, 0.0397, 0.0228, 0.0166, 0.0141, 0.0133, 0.0135, 0.0145 },
+            { 1.0, 0.0952, 0.0329, 0.0174, 0.0118, 0.00943, 0.00844, 0.00819, 0.00846 }, // 10
+            { 1.0, 0.0869, 0.0276, 0.0136, 0.00864, 0.0065, 0.00552, 0.00513, 0.00509 },
+            { 1.0, 0.08, 0.0236, 0.0108, 0.00646, 0.00459, 0.00371, 0.00329, 0.00314 },
+            { 1.0, 0.074, 0.0203, 0.00875, 0.00492, 0.00332, 0.00255, 0.00217, 0.00199, 0.00194 },
+            { 1.0, 0.0689, 0.0177, 0.00718, 0.00381, 0.00244, 0.00179, 0.00146, 0.00129, 0.00121, 0.0012 },
+            { 1.0, 0.0645, 0.0156, 0.00596, 0.003, 0.00183, 0.00128, 0.001, 0.000852, 0.000775, 0.000744 }, // 15
+            { 1.0, 0.0606, 0.0138, 0.005, 0.00239, 0.00139, 0.000935, 0.000702, 0.000574, 0.000505, 0.00047, 0.000459 },
+            { 1.0, 0.0571, 0.0123, 0.00423, 0.00193, 0.00107, 0.000692, 0.000499, 0.000394, 0.000335, 0.000302,
+                    0.000287, 0.000284 },
+            { 1.0, 0.054, 0.0111, 0.00362, 0.00158, 0.000839, 0.000519, 0.00036, 0.000275, 0.000226, 0.000198,
+                    0.000183, 0.000176 },
+            { 1.0, 0.0513, 0.00998, 0.00312, 0.0013, 0.000663, 0.000394, 0.000264, 0.000194, 0.000155, 0.000132,
+                    0.000118, 0.000111, 0.000109 },
+            { 1.0, 0.0488, 0.00906, 0.0027, 0.00108, 0.00053, 0.000303, 0.000196, 0.00014, 0.000108, 8.89e-05,
+                    7.77e-05, 7.12e-05, 6.79e-05, 6.71e-05 } // 20
+    }; // the first column is a dummy column representing K=0.
+
+    /**
+     * The optimal number of hashes for a given number of bits per element.
+     * These values are automatically calculated from the data above.
+     */
+    private static final int[] optKPerBuckets = new int[probs.length];
+
+    static {
+        for (int i = 0; i < probs.length; i++) {
+            double min = Double.MAX_VALUE;
+            double[] prob = probs[i];
+            for (int j = 0; j < prob.length; j++) {
+                if (prob[j] < min) {
+                    min = prob[j];
+                    optKPerBuckets[i] = Math.max(minK, j);
+                }
+            }
+        }
+    }
+
+    /**
+     * Given the number of buckets that can be used per element, return a
+     * specification that minimizes the false positive rate.
+     * 
+     * @param bucketsPerElement
+     *            The number of buckets per element for the filter.
+     * @return A spec that minimizes the false positive rate.
+     */
+    public static BloomFilterSpecification computeBloomSpec(int bucketsPerElement) {
+        assert bucketsPerElement >= 1;
+        assert bucketsPerElement <= probs.length - 1;
+        return new BloomFilterSpecification(optKPerBuckets[bucketsPerElement], bucketsPerElement);
+    }
+
+    /**
+     * Given a maximum tolerable false positive probability, compute a Bloom
+     * specification which will give less than the specified false positive rate,
+     * but minimize the number of buckets per element and the number of hash
+     * functions used. Because bandwidth (and therefore total bitvector size)
+     * is considered more expensive than computing power, preference is given
+     * to minimizing buckets per element rather than number of hash functions.
+     * 
+     * @param maxBucketsPerElement
+     *            The maximum number of buckets available for the filter.
+     * @param maxFalsePosProb
+     *            The maximum tolerable false positive rate.
+     * @return A Bloom Specification which would result in a false positive rate
+     *         less than specified by the function call
+     * @throws UnsupportedOperationException
+     *             if a filter satisfying the parameters cannot be met
+     */
+    public static BloomFilterSpecification computeBloomSpec(int maxBucketsPerElement, double maxFalsePosProb) {
+        assert maxBucketsPerElement >= 1;
+        assert maxBucketsPerElement <= probs.length - 1;
+        int maxK = probs[maxBucketsPerElement].length - 1;
+
+        // Handle the trivial cases
+        if (maxFalsePosProb >= probs[minBuckets][minK]) {
+            return new BloomFilterSpecification(2, optKPerBuckets[2]);
+        }
+        if (maxFalsePosProb < probs[maxBucketsPerElement][maxK]) {
+            throw new UnsupportedOperationException(String.format("Unable to satisfy %s with %s buckets per element",
+                    maxFalsePosProb, maxBucketsPerElement));
+        }
+
+        // First find the minimal required number of buckets:
+        int bucketsPerElement = 2;
+        int K = optKPerBuckets[2];
+        while (probs[bucketsPerElement][K] > maxFalsePosProb) {
+            bucketsPerElement++;
+            K = optKPerBuckets[bucketsPerElement];
+        }
+        // Now that the number of buckets is sufficient, see if we can relax K
+        // without losing too much precision.
+        while (probs[bucketsPerElement][K - 1] <= maxFalsePosProb) {
+            K--;
+        }
+
+        return new BloomFilterSpecification(K, bucketsPerElement);
+    }
+
+    /**
+     * Calculates the maximum number of buckets per element that this implementation
+     * can support. Crucially, it will lower the bucket count if necessary to meet
+     * BitSet's size restrictions.
+     */
+    public static int maxBucketsPerElement(long numElements) {
+        numElements = Math.max(1, numElements);
+        double v = Long.MAX_VALUE / (double) numElements;
+        if (v < 1.0) {
+            throw new UnsupportedOperationException("Cannot compute probabilities for " + numElements + " elements.");
+        }
+        return Math.min(BloomCalculations.probs.length - 1, (int) v);
+    }
+}
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
new file mode 100644
index 0000000..0e796b0
--- /dev/null
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
@@ -0,0 +1,269 @@
+/*
+ * 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.bloomfilter.impls;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class BloomFilter {
+
+    private final static int METADATA_PAGE_ID = 0;
+    private final static int NUM_PAGES_OFFSET = 0; // 0
+    private final static int NUM_HASHES_USED_OFFSET = NUM_PAGES_OFFSET + 4; // 4
+    private final static int NUM_ELEMENTS_OFFSET = NUM_HASHES_USED_OFFSET + 4; // 8
+    private final static int NUM_BITS_OFFSET = NUM_ELEMENTS_OFFSET + 8; // 12
+
+    private final IBufferCache bufferCache;
+    private final IFileMapProvider fileMapProvider;
+    private final FileReference file;
+    private final int[] keyFields;
+    private int fileId = -1;
+    private boolean isActivated = false;
+
+    private int numPages;
+    private int numHashes;
+    private long numElements;
+    private long numBits;
+    private int numBitsPerPage;
+
+    private final ArrayList<ICachedPage> bloomFilterPages = new ArrayList<ICachedPage>();
+    private final static long SEED = 0L;
+
+    public BloomFilter(IBufferCache bufferCache, IFileMapProvider fileMapProvider, FileReference file, int[] keyFields)
+            throws HyracksDataException {
+        this.bufferCache = bufferCache;
+        this.fileMapProvider = fileMapProvider;
+        this.file = file;
+        this.keyFields = keyFields;
+        numBitsPerPage = bufferCache.getPageSize() * Byte.SIZE;
+    }
+
+    public int getFileId() {
+        return fileId;
+    }
+
+    public FileReference getFileReference() {
+        return file;
+    }
+
+    public int getNumPages() throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("The bloom filter is not activated.");
+        }
+        return numPages;
+    }
+
+    public long getNumElements() throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("The bloom filter is not activated.");
+        }
+        return numElements;
+    }
+
+    public boolean contains(ITupleReference tuple, long[] hashes) {
+        MurmurHash128Bit.hash3_x64_128(tuple, keyFields, SEED, hashes);
+        for (int i = 0; i < numHashes; ++i) {
+            long hash = Math.abs((hashes[0] + (long) i * hashes[1]) % numBits);
+
+            ByteBuffer buffer = bloomFilterPages.get((int) (hash / numBitsPerPage)).getBuffer();
+            int byteIndex = (int) (hash % numBitsPerPage) >> 3; // divide by 8
+            byte b = buffer.get(byteIndex);
+            int bitIndex = (int) (hash % numBitsPerPage) & 0x07; // mod 8
+
+            if (!((b & (1L << bitIndex)) != 0)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private void prepareFile() throws HyracksDataException {
+        boolean fileIsMapped = false;
+        synchronized (fileMapProvider) {
+            fileIsMapped = fileMapProvider.isMapped(file);
+            if (!fileIsMapped) {
+                bufferCache.createFile(file);
+            }
+            fileId = fileMapProvider.lookupFileId(file);
+            try {
+                // Also creates the file if it doesn't exist yet.
+                bufferCache.openFile(fileId);
+            } catch (HyracksDataException e) {
+                // Revert state of buffer cache since file failed to open.
+                if (!fileIsMapped) {
+                    bufferCache.deleteFile(fileId, false);
+                }
+                throw e;
+            }
+        }
+    }
+
+    public synchronized void create() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to create the bloom filter since it is activated.");
+        }
+        prepareFile();
+        ICachedPage metaPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, METADATA_PAGE_ID), true);
+        metaPage.acquireWriteLatch();
+        metaPage.getBuffer().putInt(NUM_PAGES_OFFSET, 0);
+        metaPage.getBuffer().putInt(NUM_HASHES_USED_OFFSET, 0);
+        metaPage.getBuffer().putLong(NUM_ELEMENTS_OFFSET, 0L);
+        metaPage.getBuffer().putLong(NUM_BITS_OFFSET, 0L);
+        metaPage.releaseWriteLatch();
+        bufferCache.unpin(metaPage);
+        bufferCache.closeFile(fileId);
+    }
+
+    public synchronized void activate() throws HyracksDataException {
+        if (isActivated) {
+            return;
+        }
+
+        prepareFile();
+        readBloomFilterMetaData();
+
+        int currentPageId = 1;
+        while (currentPageId <= numPages) {
+            ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
+            bloomFilterPages.add(page);
+            ++currentPageId;
+        }
+        isActivated = true;
+    }
+
+    private void readBloomFilterMetaData() throws HyracksDataException {
+        ICachedPage metaPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, METADATA_PAGE_ID), false);
+        metaPage.acquireReadLatch();
+        numPages = metaPage.getBuffer().getInt(NUM_PAGES_OFFSET);
+        numHashes = metaPage.getBuffer().getInt(NUM_HASHES_USED_OFFSET);
+        numElements = metaPage.getBuffer().getLong(NUM_ELEMENTS_OFFSET);
+        numBits = metaPage.getBuffer().getLong(NUM_BITS_OFFSET);
+        metaPage.releaseReadLatch();
+        bufferCache.unpin(metaPage);
+    }
+
+    public synchronized void deactivate() throws HyracksDataException {
+        if (!isActivated) {
+            return;
+        }
+
+        for (int i = 0; i < numPages; ++i) {
+            bufferCache.unpin(bloomFilterPages.get(i));
+        }
+        bloomFilterPages.clear();
+        bufferCache.closeFile(fileId);
+        isActivated = false;
+    }
+
+    public synchronized void destroy() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to destroy the bloom filter since it is activated.");
+        }
+
+        file.delete();
+        if (fileId == -1) {
+            return;
+        }
+        bufferCache.deleteFile(fileId, false);
+        fileId = -1;
+    }
+
+    public IIndexBulkLoader createBuilder(long numElements, int numHashes, int numBitsPerElement)
+            throws HyracksDataException {
+        return new BloomFilterBuilder(numElements, numHashes, numBitsPerElement);
+    }
+
+    public class BloomFilterBuilder implements IIndexBulkLoader {
+        private final long[] hashes = new long[2];
+
+        private final long numElements;
+        private final int numHashes;
+        private final long numBits;
+        private final int numPages;
+
+        public BloomFilterBuilder(long numElements, int numHashes, int numBitsPerElement) throws HyracksDataException {
+            if (!isActivated) {
+                throw new HyracksDataException("Failed to create the bloom filter builder since it is not activated.");
+            }
+
+            this.numElements = numElements;
+            this.numHashes = numHashes;
+            numBits = 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) {
+                persistBloomFilterMetaData();
+                readBloomFilterMetaData();
+                int currentPageId = 1;
+                while (currentPageId <= numPages) {
+                    ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
+                    page.acquireWriteLatch();
+                    bloomFilterPages.add(page);
+                    ++currentPageId;
+                }
+            }
+        }
+
+        private void persistBloomFilterMetaData() throws HyracksDataException {
+            ICachedPage metaPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, METADATA_PAGE_ID), false);
+            metaPage.acquireWriteLatch();
+            metaPage.getBuffer().putInt(NUM_PAGES_OFFSET, numPages);
+            metaPage.getBuffer().putInt(NUM_HASHES_USED_OFFSET, numHashes);
+            metaPage.getBuffer().putLong(NUM_ELEMENTS_OFFSET, numElements);
+            metaPage.getBuffer().putLong(NUM_BITS_OFFSET, numBits);
+            metaPage.releaseWriteLatch();
+            bufferCache.unpin(metaPage);
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
+            MurmurHash128Bit.hash3_x64_128(tuple, keyFields, SEED, hashes);
+            for (int i = 0; i < numHashes; ++i) {
+                long hash = Math.abs((hashes[0] + (long) i * hashes[1]) % numBits);
+
+                ByteBuffer buffer = bloomFilterPages.get((int) (hash / numBitsPerPage)).getBuffer();
+                int byteIndex = (int) (hash % numBitsPerPage) >> 3; // divide by 8
+                byte b = buffer.get(byteIndex);
+                int bitIndex = (int) (hash % numBitsPerPage) & 0x07; // mod 8
+                b = (byte) (b | (1 << bitIndex));
+
+                buffer.put(byteIndex, b);
+            }
+        }
+
+        @Override
+        public void end() throws HyracksDataException, IndexException {
+            for (int i = 0; i < numPages; ++i) {
+                ICachedPage page = bloomFilterPages.get(i);
+                page.releaseWriteLatch();
+            }
+        }
+
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterFactory.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterFactory.java
new file mode 100644
index 0000000..d430e54
--- /dev/null
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterFactory.java
@@ -0,0 +1,41 @@
+/*
+ * 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.bloomfilter.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class BloomFilterFactory {
+    private final IBufferCache bufferCache;
+    private final IFileMapProvider fileMapProvider;
+    private final int[] bloomFilterKeyFields;
+
+    public BloomFilterFactory(IBufferCache bufferCache, IFileMapProvider fileMapProvider, int[] bloomFilterKeyFields) {
+        this.bufferCache = bufferCache;
+        this.fileMapProvider = fileMapProvider;
+        this.bloomFilterKeyFields = bloomFilterKeyFields;
+    }
+
+    public BloomFilter createBloomFiltertInstance(FileReference file) throws HyracksDataException {
+        return new BloomFilter(bufferCache, fileMapProvider, file, bloomFilterKeyFields);
+    }
+
+    public int[] getBloomFilterKeyFields() {
+        return bloomFilterKeyFields;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterSpecification.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterSpecification.java
new file mode 100644
index 0000000..a1e5517
--- /dev/null
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilterSpecification.java
@@ -0,0 +1,34 @@
+/*
+ * 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.bloomfilter.impls;
+
+public final class BloomFilterSpecification {
+    private final int numBucketsPerElement;
+    private final int numHashes;
+
+    public BloomFilterSpecification(int numBucketsPerElement, int numHashes) {
+        this.numBucketsPerElement = numBucketsPerElement;
+        this.numHashes = numHashes;
+    }
+
+    public int getNumBucketsPerElements() {
+        return numBucketsPerElement;
+    }
+
+    public int getNumHashes() {
+        return numHashes;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/MurmurHash128Bit.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/MurmurHash128Bit.java
new file mode 100644
index 0000000..0bc0a7f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/MurmurHash128Bit.java
@@ -0,0 +1,256 @@
+/*
+ * 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.bloomfilter.impls;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * The idea of this class is borrowed from http://murmurhash.googlepages.com/ and cassandra source code.
+ * We changed the hash function to operate on ITupleReference instead of a byte array.
+ **/
+public class MurmurHash128Bit {
+
+    private final static int DUMMY_FIELD = 0;
+
+    public static long rotl64(long v, int n) {
+        return ((v << n) | (v >>> (64 - n)));
+    }
+
+    public static long fmix(long k) {
+        k ^= k >>> 33;
+        k *= 0xff51afd7ed558ccdL;
+        k ^= k >>> 33;
+        k *= 0xc4ceb9fe1a85ec53L;
+        k ^= k >>> 33;
+
+        return k;
+    }
+
+    public static void hash3_x64_128(ITupleReference tuple, int[] keyFields, long seed, long[] hashes) {
+        int length = 0;
+        for (int i = 0; i < keyFields.length; ++i) {
+            length += tuple.getFieldLength(keyFields[i]);
+        }
+        final int nblocks = length >> 4; // Process as 128-bit blocks.
+
+        long h1 = seed;
+        long h2 = seed;
+
+        long c1 = 0x87c37b91114253d5L;
+        long c2 = 0x4cf5ad432745937fL;
+
+        //----------
+        // body
+
+        int currentFieldIndex = 0;
+        int bytePos = 0;
+        for (int i = 0; i < nblocks; ++i) {
+
+            long k1 = 0L;
+            for (int j = 0; j < 8; ++j) {
+                k1 += (((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos] & 0xff) << (j << 3));
+                ++bytePos;
+                if (tuple.getFieldLength(keyFields[currentFieldIndex]) == bytePos) {
+                    ++currentFieldIndex;
+                    bytePos = 0;
+                }
+            }
+            long k2 = 0L;
+            for (int j = 0; j < 8; ++j) {
+                k2 += (((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos] & 0xff) << (j << 3));
+                ++bytePos;
+                if (tuple.getFieldLength(keyFields[currentFieldIndex]) == bytePos) {
+                    ++currentFieldIndex;
+                    bytePos = 0;
+                }
+            }
+
+            k1 *= c1;
+            k1 = rotl64(k1, 31);
+            k1 *= c2;
+            h1 ^= k1;
+
+            h1 = rotl64(h1, 27);
+            h1 += h2;
+            h1 = h1 * 5 + 0x52dce729;
+
+            k2 *= c2;
+            k2 = rotl64(k2, 33);
+            k2 *= c1;
+            h2 ^= k2;
+
+            h2 = rotl64(h2, 31);
+            h2 += h1;
+            h2 = h2 * 5 + 0x38495ab5;
+        }
+
+        //----------
+        // tail
+
+        long k1 = 0L;
+        long k2 = 0L;
+
+        currentFieldIndex = keyFields.length - 1;
+        bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+        switch (length & 15) {
+            case 15:
+                k2 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 48;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 14:
+                k2 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 40;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 13:
+                k2 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 32;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 12:
+                k2 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 24;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 11:
+                k2 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 16;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 10:
+                k2 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 8;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 9:
+                k2 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]);
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+                k2 *= c2;
+                k2 = rotl64(k2, 33);
+                k2 *= c1;
+                h2 ^= k2;
+
+            case 8:
+                k1 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 56;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 7:
+                k1 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 48;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 6:
+                k1 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 40;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 5:
+                k1 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 32;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 4:
+                k1 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 24;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 3:
+                k1 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 16;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 2:
+                k1 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]) << 8;
+                --bytePos;
+                if (bytePos == -1) {
+                    --currentFieldIndex;
+                    bytePos = tuple.getFieldLength(keyFields[currentFieldIndex]) - 1;
+                }
+            case 1:
+                k1 ^= ((long) tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex])
+                        + bytePos]);
+                k1 *= c1;
+                k1 = rotl64(k1, 31);
+                k1 *= c2;
+                h1 ^= k1;
+        };
+
+        //----------
+        // finalization
+
+        h1 ^= length;
+        h2 ^= length;
+
+        h1 += h2;
+        h2 += h1;
+
+        h1 = fmix(h1);
+        h2 = fmix(h2);
+
+        h1 += h2;
+        h2 += h1;
+
+        hashes[0] = h1;
+        hashes[1] = h2;
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/pom.xml b/hyracks/hyracks-storage-am-btree/pom.xml
index f251d51..4353b41 100644
--- a/hyracks/hyracks-storage-am-btree/pom.xml
+++ b/hyracks/hyracks-storage-am-btree/pom.xml
@@ -1,8 +1,6 @@
 <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>hyracks-storage-am-btree</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-storage-am-btree</name>
 
   <parent>
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeFrame.java
index 7a61d09..ee3fd90 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeFrame.java
@@ -15,17 +15,22 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.api;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
 public interface IBTreeFrame extends ITreeIndexFrame {
-	public int findUpdateTupleIndex(ITupleReference tuple) throws TreeIndexException;
-	public int findInsertTupleIndex(ITupleReference tuple) throws TreeIndexException;	
-	public int findDeleteTupleIndex(ITupleReference tuple) throws TreeIndexException;
-	public void insertSorted(ITupleReference tuple);
-    public boolean getSmFlag();
+    public int findInsertTupleIndex(ITupleReference tuple) throws TreeIndexException;
+
+    public int findDeleteTupleIndex(ITupleReference tuple) throws TreeIndexException;
+
+    public void insertSorted(ITupleReference tuple);
+
     public void setSmFlag(boolean smFlag);
-    public void setMultiComparator(MultiComparator cmp);
+
+    public boolean getSmFlag();
+
+    public void validate(PageValidationInfo pvi) throws HyracksDataException;
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeInteriorFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeInteriorFrame.java
index 23fdcf5..ffdcc5c 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeInteriorFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeInteriorFrame.java
@@ -15,10 +15,11 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.api;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
 
 public interface IBTreeInteriorFrame extends IBTreeFrame {
-    public int getChildPageId(RangePredicate pred);
+    public int getChildPageId(RangePredicate pred) throws HyracksDataException;
 
     public int getLeftmostChildPageId();
 
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeLeafFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeLeafFrame.java
index 74bf2b0..bbb67bd 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeLeafFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeLeafFrame.java
@@ -24,13 +24,21 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
 public interface IBTreeLeafFrame extends IBTreeFrame {
+    public int findTupleIndex(ITupleReference searchKey, ITreeIndexTupleReference pageTuple, MultiComparator cmp,
+            FindTupleMode ftm, FindTupleNoExactMatchPolicy ftp) throws HyracksDataException;
+
+    public int findUpdateTupleIndex(ITupleReference tuple) throws TreeIndexException;
+
+    public int findUpsertTupleIndex(ITupleReference tuple) throws TreeIndexException;
+
+    /**
+     * @param searchTuple the tuple to match 
+     * @param targetTupleIndex the index of the tuple to check
+     * @return the tuple at targetTupleIndex if its keys match searchTuple's keys, otherwise null 
+     */
+    public ITupleReference getMatchingKeyTuple(ITupleReference searchTuple, int targetTupleIndex);
+
     public void setNextLeaf(int nextPage);
 
     public int getNextLeaf();
-
-    public int findTupleIndex(ITupleReference searchKey, ITreeIndexTupleReference pageTuple, MultiComparator cmp,
-            FindTupleMode ftm, FindTupleNoExactMatchPolicy ftp) throws HyracksDataException;
-    
-    public int findUpsertTupleIndex(ITupleReference tuple) throws TreeIndexException;
-    public ITupleReference getUpsertBeforeTuple(ITupleReference tuple, int targetTupleIndex) throws TreeIndexException;
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IPrefixSlotManager.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IPrefixSlotManager.java
index 0636968..6ec5eef 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IPrefixSlotManager.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IPrefixSlotManager.java
@@ -22,29 +22,28 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
-// a slot consists of two fields:
-// first field is 1 byte, it indicates the slot number of a prefix tuple
-// we call the first field prefixSlotOff
-// second field is 3 bytes, it points to the start offset of a tuple
-// we call the second field tupleOff
-
-// we distinguish between two slot types:
-// prefix slots that point to prefix tuples, 
-// a frame is assumed to have a field numPrefixTuples
-// tuple slots that point to data tuples
-// a frame is assumed to have a field numTuples
-// a tuple slot contains a tuple pointer and a pointer to a prefix slot (prefix slot number) 
-
-// INSERT procedure
-// a tuple insertion may use an existing prefix tuple 
-// a tuple insertion may never create a new prefix tuple
-// modifying the prefix slots would be extremely expensive because: 
-// potentially all tuples slots would have to change their prefix slot pointers
-// all prefixes are recomputed during a reorg or compaction
-
+/**
+ *  A slot consists of two fields. The first field is 1 byte and it indicates the slot number of 
+ *  a prefix tuple that is called the first field prefixSlotOff. The second field is 3 bytes and 
+ *  it points to the start offset of a tuple that is called the second field tupleOff.
+ *  
+ *  We distinguish between two slot types:
+ *      1) prefix slots that point to prefix tuples (a frame is assumed to have a field numPrefixTuples)
+ *      2) tuple slots that point to data tuples (a frame is assumed to have a field numTuples)
+ *      
+ *  A tuple slot contains a tuple pointer and a pointer to a prefix slot (prefix slot number).
+ *  
+ *  INSERT procedure:
+ *      - A tuple insertion may use an existing prefix tuple
+ *      - A tuple insertion may never create a new prefix tuple
+ *  
+ *  Modifying the prefix slots would be extremely expensive because potentially all tuples slots 
+ *  would have to change their prefix slot pointers. All prefixes are recomputed during a reorg 
+ *  or compaction.
+ */
 public interface IPrefixSlotManager extends ISlotManager {
     // TODO: Clean up interface after extending ISlotManager.
-	
+
     public int decodeFirstSlotField(int slot);
 
     public int decodeSecondSlotField(int slot);
@@ -57,7 +56,9 @@
 
     public int insertSlot(int slot, int tupleOff);
 
-    // returns prefix slot number, returns TUPLE_UNCOMPRESSED if none found
+    /** 
+     * @return the prefix slot number or FieldPrefixSlotManager.TUPLE_UNCOMPRESSED if none found
+     */
     public int findPrefix(ITupleReference tuple, ITreeIndexTupleReference framePrefixTuple);
 
     public int getTupleSlotStartOff();
@@ -78,6 +79,6 @@
 
     // functions for testing
     public void setPrefixSlot(int tupleIndex, int slot);
-    
+
     public void setMultiComparator(MultiComparator cmp);
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/ITupleAcceptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/ITupleAcceptor.java
new file mode 100644
index 0000000..5f4b30a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/ITupleAcceptor.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 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.btree.api;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ITupleAcceptor {
+    public boolean accept(ITupleReference tuple);
+}
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
index f78b6e4..7e27113 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
@@ -33,15 +33,13 @@
 
 public class FieldPrefixCompressor implements ITreeIndexFrameCompressor {
 
-    // minimum ratio of uncompressed tuples to total tuple to consider
-    // re-compression
-    private float ratioThreshold;
+    // minimum ratio of uncompressed tuples to total tuple to consider re-compression
+    private final float ratioThreshold;
 
-    // minimum number of tuple matching field prefixes to consider compressing
-    // them
-    private int occurrenceThreshold;
+    // minimum number of tuple matching field prefixes to consider compressing them
+    private final int occurrenceThreshold;
 
-    private ITypeTraits[] typeTraits;
+    private final ITypeTraits[] typeTraits;
 
     public FieldPrefixCompressor(ITypeTraits[] typeTraits, float ratioThreshold, int occurrenceThreshold) {
         this.typeTraits = typeTraits;
@@ -51,8 +49,8 @@
 
     @Override
     public boolean compress(ITreeIndexFrame indexFrame, MultiComparator cmp) throws Exception {
-        BTreeFieldPrefixNSMLeafFrame frame = (BTreeFieldPrefixNSMLeafFrame)indexFrame;
-    	int tupleCount = frame.getTupleCount();
+        BTreeFieldPrefixNSMLeafFrame frame = (BTreeFieldPrefixNSMLeafFrame) indexFrame;
+        int tupleCount = frame.getTupleCount();
         if (tupleCount <= 0) {
             frame.setPrefixTupleCount(0);
             frame.setFreeSpaceOff(frame.getOrigFreeSpaceOff());
@@ -63,7 +61,7 @@
         if (cmp.getKeyFieldCount() == 1) {
             return false;
         }
-        
+
         int uncompressedTupleCount = frame.getUncompressedTupleCount();
         float ratio = (float) uncompressedTupleCount / (float) tupleCount;
         if (ratio < ratioThreshold)
@@ -74,7 +72,7 @@
 
         ByteBuffer buf = frame.getBuffer();
         byte[] pageArray = buf.array();
-        IPrefixSlotManager slotManager = frame.slotManager;
+        IPrefixSlotManager slotManager = (IPrefixSlotManager) frame.getSlotManager();
 
         // perform analysis pass
         ArrayList<KeyPartition> keyPartitions = getKeyPartitions(frame, cmp, occurrenceThreshold);
@@ -95,8 +93,7 @@
                 }
             }
 
-            // ignore keyPartitions with no benefit and don't count bytes and
-            // slots needed
+            // ignore keyPartitions with no benefit and don't count bytes and slots needed
             if (kp.maxBenefitMinusCost <= 0)
                 continue;
 
@@ -104,12 +101,9 @@
             totalSlotsNeeded += kp.pmi[kp.maxPmiIndex].prefixSlotsNeeded;
         }
 
-        // System.out.println("TOTAL SLOTS NEEDED: " + totalSlotsNeeded);
-
         // we use a greedy heuristic to solve this "knapsack"-like problem
         // (every keyPartition has a space savings and a number of slots
-        // required, but the number of slots are constrained by
-        // MAX_PREFIX_SLOTS)
+        // required, but the number of slots are constrained by MAX_PREFIX_SLOTS)
         // we sort the keyPartitions by maxBenefitMinusCost / prefixSlotsNeeded
         // and later choose the top MAX_PREFIX_SLOTS
         int[] newPrefixSlots;
@@ -139,8 +133,7 @@
                 keyPartitions.remove(lastIndex);
             }
 
-            // re-order keyPartitions by prefix (corresponding to original
-            // order)
+            // re-order keyPartitions by prefix (corresponding to original order)
             SortByOriginalRank originalRankComparator = new SortByOriginalRank();
             Collections.sort(keyPartitions, originalRankComparator);
         } else {
@@ -150,8 +143,7 @@
         int[] newTupleSlots = new int[tupleCount];
 
         // WARNING: our hope is that compression is infrequent
-        // here we allocate a big chunk of memory to temporary hold the new,
-        // re-compressed tuple
+        // here we allocate a big chunk of memory to temporary hold the new, re-compressed tuple
         // in general it is very hard to avoid this step
         int prefixFreeSpace = frame.getOrigFreeSpaceOff();
         int tupleFreeSpace = prefixFreeSpace + totalPrefixBytes;
@@ -159,8 +151,8 @@
         ByteBuffer byteBuffer = ByteBuffer.wrap(buffer);
 
         // perform compression, and reorg
-        // we assume that the keyPartitions are sorted by the prefixes (i.e., in
-        // the logical target order)
+        // we assume that the keyPartitions are sorted by the prefixes 
+        // (i.e., in the logical target order)
         int kpIndex = 0;
         int tupleIndex = 0;
         int prefixTupleIndex = 0;
@@ -176,19 +168,13 @@
                 // beginning of keyPartition found, compress entire keyPartition
                 if (tupleIndex == keyPartitions.get(kpIndex).firstTupleIndex) {
 
-                    // number of fields we decided to use for compression of
-                    // this keyPartition
+                    // number of fields we decided to use for compression of this keyPartition
                     int fieldCountToCompress = keyPartitions.get(kpIndex).maxPmiIndex + 1;
                     int segmentStart = keyPartitions.get(kpIndex).firstTupleIndex;
                     int tuplesInSegment = 1;
 
-                    // System.out.println("PROCESSING KEYPARTITION: " + kpIndex
-                    // + " RANGE: " + keyPartitions.get(kpIndex).firstRecSlotNum
-                    // + " " + keyPartitions.get(kpIndex).lastRecSlotNum +
-                    // " FIELDSTOCOMPRESS: " + fieldCountToCompress);
-
-                    FieldPrefixTupleReference prevTuple = new FieldPrefixTupleReference(tupleWriter
-                            .createTupleReference());
+                    FieldPrefixTupleReference prevTuple = new FieldPrefixTupleReference(
+                            tupleWriter.createTupleReference());
                     prevTuple.setFieldCount(fieldCount);
 
                     FieldPrefixTupleReference tuple = new FieldPrefixTupleReference(tupleWriter.createTupleReference());
@@ -198,8 +184,7 @@
                         prevTuple.resetByTupleIndex(frame, i - 1);
                         tuple.resetByTupleIndex(frame, i);
 
-                        // check if tuples match in fieldCountToCompress of their
-                        // first fields
+                        // check if tuples match in fieldCountToCompress of their first fields
                         int prefixFieldsMatch = 0;
                         for (int j = 0; j < fieldCountToCompress; j++) {
                             if (cmps[j].compare(pageArray, prevTuple.getFieldStart(j), prevTuple.getFieldLength(j),
@@ -209,8 +194,8 @@
                                 break;
                         }
 
-                        // the two tuples must match in exactly the number of
-                        // fields we decided to compress for this keyPartition
+                        // the two tuples must match in exactly the number of fields we decided 
+                        // to compress for this keyPartition
                         int processSegments = 0;
                         if (prefixFieldsMatch == fieldCountToCompress)
                             tuplesInSegment++;
@@ -221,12 +206,10 @@
                             processSegments++;
 
                         for (int r = 0; r < processSegments; r++) {
-                            // compress current segment and then start new
-                            // segment
+                            // compress current segment and then start new segment
                             if (tuplesInSegment < occurrenceThreshold || fieldCountToCompress <= 0) {
-                                // segment does not have at least
-                                // occurrenceThreshold tuples, so write tuples
-                                // uncompressed
+                                // segment does not have at least occurrenceThreshold tuples, so 
+                                // write tuples uncompressed
                                 for (int j = 0; j < tuplesInSegment; j++) {
                                     int slotNum = segmentStart + j;
                                     tupleToWrite.resetByTupleIndex(frame, slotNum);
@@ -236,29 +219,14 @@
                                 }
                                 uncompressedTupleCount += tuplesInSegment;
                             } else {
-                                // segment has enough tuples, compress segment
-                                // extract prefix, write prefix tuple to buffer,
-                                // and set prefix slot
+                                // segment has enough tuples: compress segment, extract prefix, 
+                                // write prefix tuple to buffer, and set prefix slot
                                 newPrefixSlots[newPrefixSlots.length - 1 - prefixTupleIndex] = slotManager
                                         .encodeSlotFields(fieldCountToCompress, prefixFreeSpace);
-                                // int tmp = freeSpace;
-                                // prevRec.reset();
-                                // System.out.println("SOURCE CONTENTS: " +
-                                // buf.getInt(prevRec.getFieldOff()) + " " +
-                                // buf.getInt(prevRec.getFieldOff()+4));
                                 prefixFreeSpace += tupleWriter.writeTupleFields(prevTuple, 0, fieldCountToCompress,
                                         byteBuffer.array(), prefixFreeSpace);
-                                // System.out.println("WRITING PREFIX RECORD " +
-                                // prefixSlotNum + " AT " + tmp + " " +
-                                // freeSpace);
-                                // System.out.print("CONTENTS: ");
-                                // for(int x = 0; x < fieldCountToCompress; x++)
-                                // System.out.print(buf.getInt(tmp + x*4) +
-                                // " ");
-                                // System.out.println();
 
-                                // truncate tuples, write them to buffer, and
-                                // set tuple slots
+                                // truncate tuples, write them to buffer, and set tuple slots
                                 for (int j = 0; j < tuplesInSegment; j++) {
                                     int currTupleIndex = segmentStart + j;
                                     tupleToWrite.resetByTupleIndex(frame, currTupleIndex);
@@ -298,27 +266,24 @@
             tupleIndex++;
         }
 
-        // sanity check to see if we have written exactly as many prefix bytes
-        // as computed before
+        // sanity check to see if we have written exactly as many prefix bytes as computed before
         if (prefixFreeSpace != frame.getOrigFreeSpaceOff() + totalPrefixBytes) {
             throw new Exception("ERROR: Number of prefix bytes written don't match computed number");
         }
 
-        // in some rare instances our procedure could even increase the space
-        // requirement which is very dangerous
-        // this can happen to to the greedy solution of the knapsack-like
-        // problem
-        // therefore, we check if the new space exceeds the page size to avoid
-        // the only danger of an increasing space
+        // in some rare instances our procedure could even increase the space requirement which is very dangerous
+        // this can happen to to the greedy solution of the knapsack-like problem
+        // therefore, we check if the new space exceeds the page size to avoid the only danger of 
+        // an increasing space
         int totalSpace = tupleFreeSpace + newTupleSlots.length * slotManager.getSlotSize() + newPrefixSlots.length
                 * slotManager.getSlotSize();
         if (totalSpace > buf.capacity())
-            return false; // just leave the page as is
+            // just leave the page as is
+            return false;
 
         // copy new tuple and new slots into original page
         int freeSpaceAfterInit = frame.getOrigFreeSpaceOff();
-        System.arraycopy(buffer, freeSpaceAfterInit, pageArray, freeSpaceAfterInit, tupleFreeSpace
-                        - freeSpaceAfterInit);
+        System.arraycopy(buffer, freeSpaceAfterInit, pageArray, freeSpaceAfterInit, tupleFreeSpace - freeSpaceAfterInit);
 
         // copy prefix slots
         int slotOffRunner = buf.capacity() - slotManager.getSlotSize();
@@ -333,22 +298,6 @@
             slotOffRunner -= slotManager.getSlotSize();
         }
 
-        // int originalFreeSpaceOff = frame.getOrigFreeSpaceOff();
-        // System.out.println("ORIGINALFREESPACE: " + originalFreeSpaceOff);
-        // System.out.println("RECSPACE BEF: " + (frame.getFreeSpaceOff() -
-        // originalFreeSpaceOff));
-        // System.out.println("RECSPACE AFT: " + (recordFreeSpace -
-        // originalFreeSpaceOff));
-        // System.out.println("PREFIXSLOTS BEF: " +
-        // frame.getNumPrefixRecords());
-        // System.out.println("PREFIXSLOTS AFT: " + newPrefixSlots.length);
-        //        
-        // System.out.println("FREESPACE BEF: " + frame.getFreeSpaceOff());
-        // System.out.println("FREESPACE AFT: " + recordFreeSpace);
-        // System.out.println("PREFIXES: " + newPrefixSlots.length + " / " +
-        // FieldPrefixSlotManager.MAX_PREFIX_SLOTS);
-        // System.out.println("RECORDS: " + newRecordSlots.length);
-
         // update space fields, TODO: we need to update more fields
         frame.setFreeSpaceOff(tupleFreeSpace);
         frame.setPrefixTupleCount(newPrefixSlots.length);
@@ -377,7 +326,7 @@
         int maxCmps = cmps.length - 1;
         ByteBuffer buf = frame.getBuffer();
         byte[] pageArray = buf.array();
-        IPrefixSlotManager slotManager = frame.slotManager;
+        IPrefixSlotManager slotManager = (IPrefixSlotManager) frame.getSlotManager();
 
         ArrayList<KeyPartition> keyPartitions = new ArrayList<KeyPartition>();
         KeyPartition kp = new KeyPartition(maxCmps);
@@ -397,10 +346,6 @@
             prevTuple.resetByTupleIndex(frame, i - 1);
             tuple.resetByTupleIndex(frame, i);
 
-            // System.out.println("BEFORE RECORD: " + i + " " + rec.recSlotOff +
-            // " " + rec.recOff);
-            // kp.print();
-
             int prefixFieldsMatch = 0;
             for (int j = 0; j < maxCmps; j++) {
 
@@ -415,16 +360,15 @@
                                     - prefixFieldsMatch);
 
                     if (kp.pmi[j].matches == occurrenceThreshold) {
-                        // if we compress this prefix, we pay the cost of
-                        // storing it once, plus the size for one prefix slot
+                        // if we compress this prefix, we pay the cost of storing it once, plus 
+                        // the size for one prefix slot
                         kp.pmi[j].prefixBytes += prefixBytes;
                         kp.pmi[j].spaceCost += prefixBytes + slotManager.getSlotSize();
                         kp.pmi[j].prefixSlotsNeeded++;
                         kp.pmi[j].spaceBenefit += occurrenceThreshold * spaceBenefit;
                     } else if (kp.pmi[j].matches > occurrenceThreshold) {
-                        // we are beyond the occurrence threshold, every
-                        // additional tuple with a matching prefix increases the
-                        // benefit
+                        // we are beyond the occurrence threshold, every additional tuple with a 
+                        // matching prefix increases the benefit
                         kp.pmi[j].spaceBenefit += spaceBenefit;
                     }
                 } else {
@@ -433,15 +377,8 @@
                 }
             }
 
-            // System.out.println();
-            // System.out.println("AFTER RECORD: " + i);
-            // kp.print();
-            // System.out.println("-----------------");
-
-            // this means not even the first field matched, so we start to
-            // consider a new "key partition"
+            // this means not even the first field matched, so we start to consider a new "key partition"
             if (maxCmps > 0 && prefixFieldsMatch == 0) {
-                // System.out.println("NEW KEY PARTITION");
                 kp.lastTupleIndex = i - 1;
 
                 // remove keyPartitions that don't have enough tuples
@@ -478,7 +415,6 @@
         public int maxPmiIndex = -1;
 
         // number of fields used for compression for this kp of current page
-
         public KeyPartition(int numKeyFields) {
             pmi = new PrefixMatchInfo[numKeyFields];
             for (int i = 0; i < numKeyFields; i++) {
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelper.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelper.java
index 699915d..982f0ed 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelper.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
 
@@ -31,10 +32,11 @@
 
     @Override
     public ITreeIndex createIndexInstance() throws HyracksDataException {
+        AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
         try {
-            return BTreeUtils.createBTree(opDesc.getStorageManager().getBufferCache(ctx), opDesc
-                    .getOpCallbackProvider().getOperationCallback(), treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc
-                    .getTreeIndexComparatorFactories(), BTreeLeafFrameType.REGULAR_NSM);
+            return BTreeUtils.createBTree(opDesc.getStorageManager().getBufferCache(ctx), opDesc.getStorageManager()
+                    .getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc
+                    .getTreeIndexComparatorFactories(), BTreeLeafFrameType.REGULAR_NSM, file);
         } catch (BTreeException e) {
             throw new HyracksDataException(e);
         }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index f403312..c56308b 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -23,32 +23,34 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class BTreeSearchOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    // fields in input tuple to be used as low keys
-    protected int[] lowKeyFields;
-    // fields in input tuple to be used as high keys
-    protected int[] highKeyFields;
-    protected boolean lowKeyInclusive;
-    protected boolean highKeyInclusive;
+    protected final int[] lowKeyFields;
+    protected final int[] highKeyFields;
+    protected final boolean lowKeyInclusive;
+    protected final boolean highKeyInclusive;
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories, int[] lowKeyFields, int[] highKeyFields,
-            boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory,
-            boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, null, retainInput, opCallbackProvider);
+            IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] lowKeyFields,
+            int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
+            IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
+            ISearchOperationCallbackFactory searchOpCallbackProvider) {
+        super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, retainInput,
+                NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider,
+                NoOpOperationCallbackFactory.INSTANCE);
         this.lowKeyFields = lowKeyFields;
         this.highKeyFields = highKeyFields;
         this.lowKeyInclusive = lowKeyInclusive;
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index b284847..dc5d161 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -19,16 +19,18 @@
 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.ISearchPredicate;
+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.PermutingFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexSearchOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
 
-public class BTreeSearchOperatorNodePushable extends TreeIndexSearchOperatorNodePushable {
+public class BTreeSearchOperatorNodePushable extends IndexSearchOperatorNodePushable {
+    protected final boolean lowKeyInclusive;
+    protected final boolean highKeyInclusive;
+
     protected PermutingFrameTupleReference lowKey;
     protected PermutingFrameTupleReference highKey;
-    protected boolean lowKeyInclusive;
-    protected boolean highKeyInclusive;
     protected MultiComparator lowKeySearchCmp;
     protected MultiComparator highKeySearchCmp;
 
@@ -60,6 +62,7 @@
 
     @Override
     protected ISearchPredicate createSearchPredicate() {
+        ITreeIndex treeIndex = (ITreeIndex) index;
         lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(treeIndex.getComparatorFactories(), lowKey);
         highKeySearchCmp = BTreeUtils.getSearchMultiComparator(treeIndex.getComparatorFactories(), highKey);
         return new RangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp, highKeySearchCmp);
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
index 1fc5cb8..f13ecae 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -23,11 +23,10 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITupleUpdaterFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 public class BTreeUpdateSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
@@ -37,14 +36,15 @@
     private final ITupleUpdaterFactory tupleUpdaterFactory;
 
     public BTreeUpdateSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories, int[] lowKeyFields, int[] highKeyFields,
-            boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory,
-            boolean retainInput, IOperationCallbackProvider opCallbackProvider, ITupleUpdaterFactory tupleUpdaterFactory) {
-        super(spec, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, dataflowHelperFactory,
-                retainInput, opCallbackProvider);
+            IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] lowKeyFields,
+            int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
+            IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
+            ISearchOperationCallbackFactory searchOpCallbackProvider, ITupleUpdaterFactory tupleUpdaterFactory) {
+        super(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields, lowKeyInclusive,
+                highKeyInclusive, dataflowHelperFactory, retainInput, searchOpCallbackProvider);
         this.tupleUpdaterFactory = tupleUpdaterFactory;
     }
 
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index 1141639..648e523 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -21,7 +21,9 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITupleUpdater;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 
@@ -38,6 +40,8 @@
 
     @Override
     protected ITreeIndexCursor createCursor() {
+        ITreeIndex treeIndex = (ITreeIndex) index;
+        ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame();
         return new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);
     }
 
@@ -47,9 +51,9 @@
             tb.reset();
             cursor.next();
             if (retainInput) {
-            	frameTuple.reset(accessor, tupleIndex);
+                frameTuple.reset(accessor, tupleIndex);
                 for (int i = 0; i < frameTuple.getFieldCount(); i++) {
-                	dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
+                    dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
                     tb.addFieldEndOffset();
                 }
             }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeDuplicateKeyException.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeDuplicateKeyException.java
index d6d945f..5a1d610 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeDuplicateKeyException.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeDuplicateKeyException.java
@@ -17,11 +17,11 @@
 
 public class BTreeDuplicateKeyException extends BTreeException {
     private static final long serialVersionUID = 1L;
-    
+
     public BTreeDuplicateKeyException(Exception e) {
         super(e);
     }
-    
+
     public BTreeDuplicateKeyException(String message) {
         super(message);
     }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeNonExistentKeyException.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeNonExistentKeyException.java
index 81a0e79..989f118 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeNonExistentKeyException.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeNonExistentKeyException.java
@@ -16,13 +16,13 @@
 package edu.uci.ics.hyracks.storage.am.btree.exceptions;
 
 public class BTreeNonExistentKeyException extends BTreeException {
-    
+
     private static final long serialVersionUID = 1L;
-    
+
     public BTreeNonExistentKeyException(Exception e) {
         super(e);
     }
-    
+
     public BTreeNonExistentKeyException(String message) {
         super(message);
     }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeNotUpdateableException.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeNotUpdateableException.java
index 73b22d8..7e83c69 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeNotUpdateableException.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeNotUpdateableException.java
@@ -17,11 +17,11 @@
 
 public class BTreeNotUpdateableException extends BTreeException {
     private static final long serialVersionUID = 1L;
-    
+
     public BTreeNotUpdateableException(Exception e) {
         super(e);
     }
-    
+
     public BTreeNotUpdateableException(String message) {
         super(message);
     }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
index fb2e833..354aa1e 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
@@ -27,10 +27,10 @@
 import edu.uci.ics.hyracks.storage.am.btree.compressors.FieldPrefixCompressor;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeNonExistentKeyException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
 import edu.uci.ics.hyracks.storage.am.btree.impls.FieldPrefixPrefixTupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.impls.FieldPrefixSlotManager;
 import edu.uci.ics.hyracks.storage.am.btree.impls.FieldPrefixTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.ISlotManager;
 import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameCompressor;
@@ -45,8 +45,9 @@
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 
-// WARNING: only works when tupleWriter is an instance of TypeAwareTupleWriter
-
+/**
+ * WARNING: only works when tupleWriter is an instance of TypeAwareTupleWriter
+ */
 public class BTreeFieldPrefixNSMLeafFrame implements IBTreeLeafFrame {
 
     protected static final int pageLsnOff = 0; // 0
@@ -57,28 +58,26 @@
     protected static final int smFlagOff = levelOff + 1; // 21
     protected static final int uncompressedTupleCountOff = smFlagOff + 1; // 22
     protected static final int prefixTupleCountOff = uncompressedTupleCountOff + 4; // 26
-
     protected static final int nextLeafOff = prefixTupleCountOff + 4; // 30
 
+    private final IPrefixSlotManager slotManager;
+    private final ITreeIndexFrameCompressor compressor;
+    private final FieldPrefixTupleReference frameTuple;
+    private final FieldPrefixPrefixTupleReference framePrefixTuple;
+    private final ITreeIndexTupleWriter tupleWriter;
+
+    private MultiComparator cmp;
+
     protected ICachedPage page = null;
     protected ByteBuffer buf = null;
 
-    public final ITreeIndexFrameCompressor compressor;
-    // TODO: Should be protected, but will trigger some refactoring.
-    public final IPrefixSlotManager slotManager;
-
-    private final ITreeIndexTupleWriter tupleWriter;
-    private MultiComparator cmp;
-    
-    private final FieldPrefixTupleReference frameTuple;
-    private final FieldPrefixPrefixTupleReference framePrefixTuple;
-
     public BTreeFieldPrefixNSMLeafFrame(ITreeIndexTupleWriter tupleWriter) {
         this.tupleWriter = tupleWriter;
         this.frameTuple = new FieldPrefixTupleReference(tupleWriter.createTupleReference());
+        this.slotManager = new FieldPrefixSlotManager();
+
         ITypeTraits[] typeTraits = ((TypeAwareTupleWriter) tupleWriter).getTypeTraits();
         this.framePrefixTuple = new FieldPrefixPrefixTupleReference(typeTraits);
-        this.slotManager = new FieldPrefixSlotManager();
         this.compressor = new FieldPrefixCompressor(typeTraits, 0.001f, 2);
     }
 
@@ -108,19 +107,18 @@
         }
     }
 
-    // assumptions:
-    // 1. prefix tuple are stored contiguously
-    // 2. prefix tuple are located before tuples (physically on the page)
-    // 3. prefix tuple are sorted (last prefix tuple is at highest offset)
-    // this procedure will not move prefix tuples
+    // Assumptions:
+    // 1) prefix tuples are stored contiguously
+    // 2) prefix tuples are located before tuples (physically on the page)
+    // 3) prefix tuples are sorted (last prefix tuple is at highest offset)
+    // This procedure will not move prefix tuples.
     @Override
     public boolean compact() {
         resetSpaceParams();
 
         int tupleCount = buf.getInt(tupleCountOff);
 
-        // determine start of target free space (depends on assumptions stated
-        // above)
+        // determine start of target free space (depends on assumptions stated above)
         int freeSpace = buf.getInt(freeSpaceOff);
         int prefixTupleCount = buf.getInt(prefixTupleCountOff);
         if (prefixTupleCount > 0) {
@@ -185,8 +183,7 @@
         int length = tupleSlotOff - slotEndOff;
         System.arraycopy(buf.array(), slotEndOff, buf.array(), slotEndOff + slotManager.getSlotSize(), length);
 
-        // maintain space information, get size of tuple suffix (suffix
-        // could be entire tuple)
+        // maintain space information, get size of tuple suffix (suffix could be entire tuple)
         int tupleSize = 0;
         int suffixFieldStart = 0;
         if (prefixSlotNum == FieldPrefixSlotManager.TUPLE_UNCOMPRESSED) {
@@ -257,37 +254,37 @@
         buf.putInt(freeSpaceOff, buf.getInt(freeSpaceOff) + bytesWritten);
         buf.putInt(totalFreeSpaceOff, buf.getInt(totalFreeSpaceOff) - bytesWritten - slotManager.getSlotSize());
     }
-    
+
     @Override
     public FrameOpSpaceStatus hasSpaceUpdate(ITupleReference newTuple, int oldTupleIndex) {
         int tupleIndex = slotManager.decodeSecondSlotField(oldTupleIndex);
         frameTuple.resetByTupleIndex(this, tupleIndex);
-        
+
         int oldTupleBytes = 0;
         int newTupleBytes = 0;
-        
+
         int numPrefixFields = frameTuple.getNumPrefixFields();
         int fieldCount = frameTuple.getFieldCount();
         if (numPrefixFields != 0) {
-            // Check the space requirements for updating the suffix of the original tuple.            
+            // Check the space requirements for updating the suffix of the original tuple.
             oldTupleBytes = frameTuple.getSuffixTupleSize();
-            newTupleBytes = tupleWriter.bytesRequired(newTuple, numPrefixFields, fieldCount - numPrefixFields); 
+            newTupleBytes = tupleWriter.bytesRequired(newTuple, numPrefixFields, fieldCount - numPrefixFields);
         } else {
             // The original tuple is uncompressed.
             oldTupleBytes = frameTuple.getTupleSize();
             newTupleBytes = tupleWriter.bytesRequired(newTuple);
         }
-        
+
         int additionalBytesRequired = newTupleBytes - oldTupleBytes;
         // Enough space for an in-place update?
         if (additionalBytesRequired <= 0) {
             return FrameOpSpaceStatus.SUFFICIENT_INPLACE_SPACE;
         }
-        
+
         int freeContiguous = buf.capacity() - buf.getInt(freeSpaceOff)
                 - ((buf.getInt(tupleCountOff) + buf.getInt(prefixTupleCountOff)) * slotManager.getSlotSize());
-        
-        // Enough space if we delete the old tuple and insert the new one without compaction? 
+
+        // Enough space if we delete the old tuple and insert the new one without compaction?
         if (newTupleBytes <= freeContiguous) {
             return FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE;
         }
@@ -304,21 +301,24 @@
         int tupleSlotOff = slotManager.getTupleSlotOff(tupleIndex);
         int tupleSlot = buf.getInt(tupleSlotOff);
         int prefixSlotNum = slotManager.decodeFirstSlotField(tupleSlot);
-        int suffixTupleStartOff = slotManager.decodeSecondSlotField(tupleSlot);                
-        
+        int suffixTupleStartOff = slotManager.decodeSecondSlotField(tupleSlot);
+
         frameTuple.resetByTupleIndex(this, tupleIndex);
         int fieldCount = frameTuple.getFieldCount();
         int numPrefixFields = frameTuple.getNumPrefixFields();
         int oldTupleBytes = frameTuple.getSuffixTupleSize();
-        int bytesWritten = 0;        
-        
+        int bytesWritten = 0;
+
         if (inPlace) {
             // Overwrite the old tuple suffix in place.
-            bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, fieldCount - numPrefixFields, buf.array(), suffixTupleStartOff);
+            bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, fieldCount - numPrefixFields,
+                    buf.array(), suffixTupleStartOff);
         } else {
-            // Insert the new tuple suffix at the end of the free space, and change the slot value (effectively "deleting" the old tuple).
+            // Insert the new tuple suffix at the end of the free space, and change
+            // the slot value (effectively "deleting" the old tuple).
             int newSuffixTupleStartOff = buf.getInt(freeSpaceOff);
-            bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, fieldCount - numPrefixFields, buf.array(), newSuffixTupleStartOff);
+            bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, fieldCount - numPrefixFields,
+                    buf.array(), newSuffixTupleStartOff);
             // Update slot value using the same prefix slot num.
             slotManager.setSlot(tupleSlotOff, slotManager.encodeSlotFields(prefixSlotNum, newSuffixTupleStartOff));
             // Update contiguous free space pointer.
@@ -326,7 +326,7 @@
         }
         buf.putInt(totalFreeSpaceOff, buf.getInt(totalFreeSpaceOff) + oldTupleBytes - bytesWritten);
     }
-    
+
     protected void resetSpaceParams() {
         buf.putInt(freeSpaceOff, getOrigFreeSpaceOff());
         buf.putInt(totalFreeSpaceOff, getOrigTotalFreeSpace());
@@ -355,8 +355,8 @@
 
     @Override
     public int findInsertTupleIndex(ITupleReference tuple) throws TreeIndexException {
-    	int slot = slotManager.findSlot(tuple, frameTuple, framePrefixTuple, cmp, FindTupleMode.EXCLUSIVE_ERROR_IF_EXISTS,
-                FindTupleNoExactMatchPolicy.HIGHER_KEY);
+        int slot = slotManager.findSlot(tuple, frameTuple, framePrefixTuple, cmp,
+                FindTupleMode.EXCLUSIVE_ERROR_IF_EXISTS, FindTupleNoExactMatchPolicy.HIGHER_KEY);
         int tupleIndex = slotManager.decodeSecondSlotField(slot);
         // Error indicator is set if there is an exact match.
         if (tupleIndex == slotManager.getErrorIndicator()) {
@@ -364,7 +364,7 @@
         }
         return slot;
     }
-    
+
     @Override
     public int findUpsertTupleIndex(ITupleReference tuple) throws TreeIndexException {
         int slot = slotManager.findSlot(tuple, frameTuple, framePrefixTuple, cmp, FindTupleMode.INCLUSIVE,
@@ -376,15 +376,15 @@
         }
         return slot;
     }
-    
+
     @Override
-    public ITupleReference getUpsertBeforeTuple(ITupleReference tuple, int targetTupleIndex) throws TreeIndexException {
+    public ITupleReference getMatchingKeyTuple(ITupleReference searchTuple, int targetTupleIndex) {
         int tupleIndex = slotManager.decodeSecondSlotField(targetTupleIndex);
         // Examine the tuple index to determine whether it is valid or not.
         if (tupleIndex != slotManager.getGreatestKeyIndicator()) {
             // We need to check the key to determine whether it's an insert or an update.
             frameTuple.resetByTupleIndex(this, tupleIndex);
-            if (cmp.compare(tuple, frameTuple) == 0) {
+            if (cmp.compare(searchTuple, frameTuple) == 0) {
                 // The keys match, it's an update.
                 return frameTuple;
             }
@@ -393,7 +393,7 @@
         // In those cases, we are definitely dealing with an insert.
         return null;
     }
-    
+
     @Override
     public int findUpdateTupleIndex(ITupleReference tuple) throws TreeIndexException {
         int slot = slotManager.findSlot(tuple, frameTuple, framePrefixTuple, cmp, FindTupleMode.EXACT,
@@ -402,10 +402,10 @@
         // Error indicator is set if there is no exact match.
         if (tupleIndex == slotManager.getErrorIndicator()) {
             throw new BTreeNonExistentKeyException("Trying to update a tuple with a nonexistent key in leaf node.");
-        }    
+        }
         return slot;
     }
-    
+
     @Override
     public int findDeleteTupleIndex(ITupleReference tuple) throws TreeIndexException {
         int slot = slotManager.findSlot(tuple, frameTuple, framePrefixTuple, cmp, FindTupleMode.EXACT,
@@ -414,10 +414,10 @@
         // Error indicator is set if there is no exact match.
         if (tupleIndex == slotManager.getErrorIndicator()) {
             throw new BTreeNonExistentKeyException("Trying to delete a tuple with a nonexistent key in leaf node.");
-        }    
+        }
         return slot;
     }
-    
+
     @Override
     public String printHeader() {
         StringBuilder strBuilder = new StringBuilder();
@@ -438,8 +438,8 @@
         return buf.getInt(tupleCountOff);
     }
 
-    public ISlotManager getSlotManager() {
-        return null;
+    public IPrefixSlotManager getSlotManager() {
+        return slotManager;
     }
 
     @Override
@@ -537,16 +537,16 @@
     }
 
     @Override
-    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey)
-    		throws TreeIndexException {
+    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) {
 
-        BTreeFieldPrefixNSMLeafFrame rf = (BTreeFieldPrefixNSMLeafFrame)rightFrame;
+        BTreeFieldPrefixNSMLeafFrame rf = (BTreeFieldPrefixNSMLeafFrame) rightFrame;
 
         ByteBuffer right = rf.getBuffer();
         int tupleCount = getTupleCount();
         int prefixTupleCount = getPrefixTupleCount();
 
-        // Find split point, and determine into which frame the new tuple should be inserted into.
+        // Find split point, and determine into which frame the new tuple should
+        // be inserted into.
         int tuplesToLeft;
         int midSlotNum = tupleCount / 2;
         ITreeIndexFrame targetFrame = null;
@@ -576,8 +576,7 @@
             }
         }
 
-        // if we are splitting in the middle of a prefix both pages need to have
-        // the prefix slot and tuple
+        // if we are splitting in the middle of a prefix both pages need to have the prefix slot and tuple
         int boundaryTupleSlotOff = rf.slotManager.getTupleSlotOff(tuplesToLeft - 1);
         int boundaryTupleSlot = buf.getInt(boundaryTupleSlotOff);
         int boundaryPrefixSlotNum = rf.slotManager.decodeFirstSlotField(boundaryTupleSlot);
@@ -587,8 +586,7 @@
             prefixesToLeft++; // tuples on both pages share one prefix
         }
 
-        // move prefix tuples on right page to beginning of page and adjust
-        // prefix slots
+        // move prefix tuples on right page to beginning of page and adjust prefix slots
         if (prefixesToRight > 0 && prefixesToLeft > 0 && prefixTupleCount > 1) {
 
             int freeSpace = rf.getOrigFreeSpaceOff();
@@ -652,7 +650,13 @@
         rightFrame.compact();
 
         // insert last key
-        int targetTupleIndex = ((IBTreeLeafFrame)targetFrame).findInsertTupleIndex(tuple);
+        int targetTupleIndex;
+        // it's safe to catch this exception since it will have been caught before reaching here
+        try {
+            targetTupleIndex = ((IBTreeLeafFrame) targetFrame).findInsertTupleIndex(tuple);
+        } catch (TreeIndexException e) {
+            throw new IllegalStateException(e);
+        }
         targetFrame.insert(tuple, targetTupleIndex);
 
         // set split key to be highest value in left page
@@ -716,7 +720,8 @@
         int slot = slotManager.findSlot(searchKey, pageTuple, framePrefixTuple, cmp, ftm, ftp);
         int tupleIndex = slotManager.decodeSecondSlotField(slot);
         // TODO: Revisit this one. Maybe there is a cleaner way to solve this in the RangeSearchCursor.
-        if (tupleIndex == FieldPrefixSlotManager.GREATEST_KEY_INDICATOR || tupleIndex == FieldPrefixSlotManager.ERROR_INDICATOR)
+        if (tupleIndex == FieldPrefixSlotManager.GREATEST_KEY_INDICATOR
+                || tupleIndex == FieldPrefixSlotManager.ERROR_INDICATOR)
             return -1;
         else
             return tupleIndex;
@@ -727,9 +732,14 @@
         return nextLeafOff;
     }
 
-	@Override
-	public void setMultiComparator(MultiComparator cmp) {
-		this.cmp = cmp;
-		this.slotManager.setMultiComparator(cmp);
-	}
+    @Override
+    public void setMultiComparator(MultiComparator cmp) {
+        this.cmp = cmp;
+        this.slotManager.setMultiComparator(cmp);
+    }
+
+    @Override
+    public void validate(PageValidationInfo pvi) {
+        // Do nothing
+    }
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrameFactory.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrameFactory.java
index 5defb27..79d2f3a 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrameFactory.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrameFactory.java
@@ -22,8 +22,9 @@
 public class BTreeFieldPrefixNSMLeafFrameFactory implements ITreeIndexFrameFactory {
 
     private static final long serialVersionUID = 1L;
+
     private final ITreeIndexTupleWriterFactory tupleWriterFactory;
-    
+
     public BTreeFieldPrefixNSMLeafFrameFactory(ITreeIndexTupleWriterFactory tupleWriterFactory) {
         this.tupleWriterFactory = tupleWriterFactory;
     }
@@ -35,6 +36,6 @@
 
     @Override
     public ITreeIndexTupleWriterFactory getTupleWriterFactory() {
-       return tupleWriterFactory;
+        return tupleWriterFactory;
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeLeafFrameType.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeLeafFrameType.java
index 6ff44be..bd543f8 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeLeafFrameType.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeLeafFrameType.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2010 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.btree.frames;
 
 public enum BTreeLeafFrameType {
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
index d2cb2c6..90b167f 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
@@ -19,9 +19,11 @@
 import java.util.ArrayList;
 import java.util.Collections;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
@@ -41,11 +43,14 @@
     private static final int childPtrSize = 4;
 
     private final ITreeIndexTupleReference cmpFrameTuple;
+    private final ITreeIndexTupleReference previousFt;
+
     private MultiComparator cmp;
 
     public BTreeNSMInteriorFrame(ITreeIndexTupleWriter tupleWriter) {
         super(tupleWriter, new OrderedSlotManager());
         cmpFrameTuple = tupleWriter.createTupleReference();
+        previousFt = tupleWriter.createTupleReference();
     }
 
     @Override
@@ -59,7 +64,7 @@
         return slotManager.findTupleIndex(tuple, frameTuple, cmp, FindTupleMode.INCLUSIVE,
                 FindTupleNoExactMatchPolicy.HIGHER_KEY);
     }
-    
+
     @Override
     public FrameOpSpaceStatus hasSpaceInsert(ITupleReference tuple) {
         // Tuple bytes + child pointer + slot.
@@ -89,12 +94,9 @@
             System.arraycopy(tuple.getFieldData(tuple.getFieldCount() - 1), getLeftChildPageOff(tuple) + childPtrSize,
                     buf.array(), rightLeafOff, childPtrSize);
         } else {
-            // If slotOff has a right (slot-)neighbor then update its child
-            // pointer.
-            // The only time when this is NOT the case, is when this is the
-            // very first tuple (or when the splitkey goes into the rightmost
-            // slot but that
-            // case is handled in the if above).
+            // If slotOff has a right (slot-)neighbor then update its child pointer.
+            // The only time when this is NOT the case, is when this is the very first tuple
+            // (or when the splitkey goes into the rightmost slot but that case is handled in the if above).
             if (buf.getInt(tupleCountOff) > 1) {
                 int rightNeighborOff = slotOff - slotManager.getSlotSize();
                 frameTuple.resetByTupleOffset(buf, slotManager.getTupleOff(rightNeighborOff));
@@ -103,7 +105,7 @@
             }
         }
     }
-    
+
     @Override
     public int findDeleteTupleIndex(ITupleReference tuple) throws TreeIndexException {
         return slotManager.findTupleIndex(tuple, frameTuple, cmp, FindTupleMode.INCLUSIVE,
@@ -135,7 +137,7 @@
         buf.putInt(totalFreeSpaceOff,
                 buf.getInt(totalFreeSpaceOff) + keySize + childPtrSize + slotManager.getSlotSize());
     }
-    
+
     @Override
     public void deleteGreatest() {
         int slotOff = slotManager.getSlotEndOff();
@@ -152,16 +154,11 @@
             buf.putInt(freeSpace, freeSpace - (keySize + childPtrSize));
         }
     }
-    
+
     @Override
     public FrameOpSpaceStatus hasSpaceUpdate(ITupleReference tuple, int oldTupleIndex) {
         throw new UnsupportedOperationException("Cannot update tuples in interior node.");
     }
-    
-    @Override
-    public int findUpdateTupleIndex(ITupleReference tuple) throws TreeIndexException {
-        throw new UnsupportedOperationException("Cannot update tuples in interior node.");
-    }
 
     @Override
     public void insertSorted(ITupleReference tuple) {
@@ -179,18 +176,31 @@
     }
 
     @Override
-    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) throws TreeIndexException {
+    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) {
         ByteBuffer right = rightFrame.getBuffer();
         int tupleCount = getTupleCount();
-        
+
         // Find split point, and determine into which frame the new tuple should be inserted into.
-        int tuplesToLeft = (tupleCount / 2) + (tupleCount % 2);
+        int tuplesToLeft;
         ITreeIndexFrame targetFrame = null;
-        frameTuple.resetByTupleIndex(this, tuplesToLeft - 1);
-        if (cmp.compare(tuple, frameTuple) <= 0) {
-            targetFrame = this;
-        } else {
+
+        int totalSize = 0;
+        int halfPageSize = buf.capacity() / 2 - getPageHeaderSize();
+        int i;
+        for (i = 0; i < tupleCount; ++i) {
+            frameTuple.resetByTupleIndex(this, i);
+            totalSize += tupleWriter.bytesRequired(frameTuple) + childPtrSize + slotManager.getSlotSize();
+            if (totalSize >= halfPageSize) {
+                break;
+            }
+        }
+
+        if (cmp.compare(tuple, frameTuple) > 0) {
+            tuplesToLeft = i;
             targetFrame = rightFrame;
+        } else {
+            tuplesToLeft = i + 1;
+            targetFrame = this;
         }
         int tuplesToRight = tupleCount - tuplesToLeft;
 
@@ -232,8 +242,13 @@
         compact();
 
         // Insert the saved split key.
-        int targetTupleIndex = ((BTreeNSMInteriorFrame) targetFrame)
-                .findInsertTupleIndex(savedSplitKey.getTuple());
+        int targetTupleIndex;
+        // it's safe to catch this exception since it will have been caught before reaching here
+        try {
+            targetTupleIndex = ((BTreeNSMInteriorFrame) targetFrame).findInsertTupleIndex(savedSplitKey.getTuple());
+        } catch (TreeIndexException e) {
+            throw new IllegalStateException(e);
+        }
         targetFrame.insert(savedSplitKey.getTuple(), targetTupleIndex);
     }
 
@@ -270,17 +285,17 @@
     }
 
     @Override
-    public int getChildPageId(RangePredicate pred) {
+    public int getChildPageId(RangePredicate pred) throws HyracksDataException {
         // Trivial case where there is only a child pointer (and no key).
         if (buf.getInt(tupleCountOff) == 0) {
             return buf.getInt(rightLeafOff);
         }
-        // Trivial cases where no low key or high key was given (e.g.
-        // during an index scan).
+        // Trivial cases where no low key or high key was given (e.g. during an
+        // index scan).
         ITupleReference tuple = null;
         FindTupleMode fsm = null;
         // The target comparator may be on a prefix of the BTree key fields.
-        MultiComparator targetCmp = pred.getLowKeyComparator();;
+        MultiComparator targetCmp = pred.getLowKeyComparator();
         tuple = pred.getLowKey();
         if (tuple == null) {
             return getLeftmostChildPageId();
@@ -322,6 +337,7 @@
         slotOff -= slotManager.getSlotSize();
         frameTuple.resetByTupleOffset(buf, slotManager.getTupleOff(slotOff));
         int childPageOff = getLeftChildPageOff(frameTuple);
+
         return buf.getInt(childPageOff);
     }
 
@@ -351,7 +367,7 @@
 
     @Override
     public int getPageHeaderSize() {
-        return rightLeafOff;
+        return rightLeafOff + 4;
     }
 
     private int getLeftChildPageOff(ITupleReference tuple) {
@@ -377,15 +393,16 @@
         this.cmp = cmp;
         cmpFrameTuple.setFieldCount(cmp.getKeyFieldCount());
         frameTuple.setFieldCount(cmp.getKeyFieldCount());
+        previousFt.setFieldCount(cmp.getKeyFieldCount());
     }
-    
+
     @Override
     public ITreeIndexTupleReference createTupleReference() {
         ITreeIndexTupleReference tuple = tupleWriter.createTupleReference();
         tuple.setFieldCount(cmp.getKeyFieldCount());
         return tuple;
     }
-    
+
     // For debugging.
     public ArrayList<Integer> getChildren(MultiComparator cmp) {
         ArrayList<Integer> ret = new ArrayList<Integer>();
@@ -407,4 +424,23 @@
         }
         return ret;
     }
+
+    public void validate(PageValidationInfo pvi) throws HyracksDataException {
+        int tupleCount = getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            frameTuple.resetByTupleIndex(this, i);
+            if (!pvi.isLowRangeNull) {
+                assert cmp.compare(pvi.lowRangeTuple, frameTuple) < 0;
+            }
+
+            if (!pvi.isHighRangeNull) {
+                assert cmp.compare(pvi.highRangeTuple, frameTuple) >= 0;
+            }
+
+            if (i > 0) {
+                previousFt.resetByTupleIndex(this, i - 1);
+                assert cmp.compare(previousFt, frameTuple) < 0;
+            }
+        }
+    }
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrameFactory.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrameFactory.java
index 8618df8..1491b55 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrameFactory.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrameFactory.java
@@ -22,6 +22,7 @@
 public class BTreeNSMInteriorFrameFactory implements ITreeIndexFrameFactory {
 
     private static final long serialVersionUID = 1L;
+
     private final ITreeIndexTupleWriterFactory tupleWriterFactory;
 
     public BTreeNSMInteriorFrameFactory(ITreeIndexTupleWriterFactory tupleWriterFactory) {
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
index 4b7f44b..04b3077 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
@@ -17,10 +17,12 @@
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeNonExistentKeyException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
 import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
@@ -33,10 +35,14 @@
 
 public class BTreeNSMLeafFrame extends TreeIndexNSMFrame implements IBTreeLeafFrame {
     protected static final int nextLeafOff = smFlagOff + 1;
+
     private MultiComparator cmp;
-    
+
+    private final ITreeIndexTupleReference previousFt;
+
     public BTreeNSMLeafFrame(ITreeIndexTupleWriter tupleWriter) {
         super(tupleWriter, new OrderedSlotManager());
+        previousFt = tupleWriter.createTupleReference();
     }
 
     @Override
@@ -65,7 +71,7 @@
         }
         return tupleIndex;
     }
-    
+
     @Override
     public int findUpdateTupleIndex(ITupleReference tuple) throws TreeIndexException {
         int tupleIndex = slotManager.findTupleIndex(tuple, frameTuple, cmp, FindTupleMode.EXACT,
@@ -73,34 +79,37 @@
         // Error indicator is set if there is no exact match.
         if (tupleIndex == slotManager.getErrorIndicator() || tupleIndex == slotManager.getGreatestKeyIndicator()) {
             throw new BTreeNonExistentKeyException("Trying to update a tuple with a nonexistent key in leaf node.");
-        }        
+        }
         return tupleIndex;
     }
-    
+
     @Override
     public int findUpsertTupleIndex(ITupleReference tuple) throws TreeIndexException {
         int tupleIndex = slotManager.findTupleIndex(tuple, frameTuple, cmp, FindTupleMode.INCLUSIVE,
                 FindTupleNoExactMatchPolicy.HIGHER_KEY);
-        // Just return the found tupleIndex. The caller will make the final decision whether to insert or update.
+        // Just return the found tupleIndex. The caller will make the final
+        // decision whether to insert or update.
         return tupleIndex;
     }
-    
+
     @Override
-    public ITupleReference getUpsertBeforeTuple(ITupleReference tuple, int targetTupleIndex) throws TreeIndexException {
+    public ITupleReference getMatchingKeyTuple(ITupleReference searchTuple, int targetTupleIndex) {
         // Examine the tuple index to determine whether it is valid or not.
         if (targetTupleIndex != slotManager.getGreatestKeyIndicator()) {
-            // We need to check the key to determine whether it's an insert or an update.
+            // We need to check the key to determine whether it's an insert or
+            // an update/delete
             frameTuple.resetByTupleIndex(this, targetTupleIndex);
-            if (cmp.compare(tuple, frameTuple) == 0) {
-                // The keys match, it's an update.
+            if (cmp.compare(searchTuple, frameTuple) == 0) {
+                // The keys match, it's an update/delete
                 return frameTuple;
             }
         }
-        // Either the tuple index is a special indicator, or the keys don't match.
+        // Either the tuple index is a special indicator, or the keys don't
+        // match.
         // In those cases, we are definitely dealing with an insert.
         return null;
     }
-    
+
     @Override
     public int findDeleteTupleIndex(ITupleReference tuple) throws TreeIndexException {
         int tupleIndex = slotManager.findTupleIndex(tuple, frameTuple, cmp, FindTupleMode.EXACT,
@@ -108,7 +117,7 @@
         // Error indicator is set if there is no exact match.
         if (tupleIndex == slotManager.getErrorIndicator() || tupleIndex == slotManager.getGreatestKeyIndicator()) {
             throw new BTreeNonExistentKeyException("Trying to delete a tuple with a nonexistent key in leaf node.");
-        }        
+        }
         return tupleIndex;
     }
 
@@ -128,21 +137,30 @@
     }
 
     @Override
-    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) throws TreeIndexException {
-    	ByteBuffer right = rightFrame.getBuffer();
-        int tupleCount = getTupleCount();        
-        
-        // Find split point, and determine into which frame the new tuple should be inserted into.
+    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) {
+        ByteBuffer right = rightFrame.getBuffer();
+        int tupleCount = getTupleCount();
+
+        // Find split point, and determine into which frame the new tuple should
+        // be inserted into.
         int tuplesToLeft;
-        int mid = tupleCount / 2;
         ITreeIndexFrame targetFrame = null;
-        int tupleOff = slotManager.getTupleOff(slotManager.getSlotEndOff() + slotManager.getSlotSize() * mid);
-        frameTuple.resetByTupleOffset(buf, tupleOff);
+        int totalSize = 0;
+        int halfPageSize = buf.capacity() / 2 - getPageHeaderSize();
+        int i;
+        for (i = 0; i < tupleCount; ++i) {
+            frameTuple.resetByTupleIndex(this, i);
+            totalSize += tupleWriter.getCopySpaceRequired(frameTuple) + slotManager.getSlotSize();
+            if (totalSize >= halfPageSize) {
+                break;
+            }
+        }
+
         if (cmp.compare(tuple, frameTuple) >= 0) {
-            tuplesToLeft = mid + (tupleCount % 2);
+            tuplesToLeft = i + 1;
             targetFrame = rightFrame;
         } else {
-            tuplesToLeft = mid;
+            tuplesToLeft = i;
             targetFrame = this;
         }
         int tuplesToRight = tupleCount - tuplesToLeft;
@@ -166,11 +184,18 @@
         compact();
 
         // Insert the new tuple.
-        int targetTupleIndex = ((BTreeNSMLeafFrame)targetFrame).findInsertTupleIndex(tuple);
+        int targetTupleIndex;
+        // it's safe to catch this exception since it will have been caught
+        // before reaching here
+        try {
+            targetTupleIndex = ((BTreeNSMLeafFrame) targetFrame).findInsertTupleIndex(tuple);
+        } catch (TreeIndexException e) {
+            throw new IllegalStateException(e);
+        }
         targetFrame.insert(tuple, targetTupleIndex);
 
         // Set the split key to be highest key in the left page.
-        tupleOff = slotManager.getTupleOff(slotManager.getSlotEndOff());
+        int tupleOff = slotManager.getTupleOff(slotManager.getSlotEndOff());
         frameTuple.resetByTupleOffset(buf, tupleOff);
         int splitKeySize = tupleWriter.bytesRequired(frameTuple, 0, cmp.getKeyFieldCount());
         splitKey.initData(splitKeySize);
@@ -197,7 +222,7 @@
 
     @Override
     public int getPageHeaderSize() {
-        return nextLeafOff;
+        return nextLeafOff + 4;
     }
 
     @Override
@@ -213,9 +238,28 @@
             buf.put(smFlagOff, (byte) 0);
         }
     }
-    
-	@Override
-	public void setMultiComparator(MultiComparator cmp) {
-		this.cmp = cmp;
-	}
-}
+
+    @Override
+    public void setMultiComparator(MultiComparator cmp) {
+        this.cmp = cmp;
+    }
+
+    public void validate(PageValidationInfo pvi) throws HyracksDataException {
+        int tupleCount = getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            frameTuple.resetByTupleIndex(this, i);
+            if (!pvi.isLowRangeNull) {
+                assert cmp.compare(pvi.lowRangeTuple, frameTuple) < 0;
+            }
+
+            if (!pvi.isHighRangeNull) {
+                assert cmp.compare(pvi.highRangeTuple, frameTuple) >= 0;
+            }
+
+            if (i > 0) {
+                previousFt.resetByTupleIndex(this, i - 1);
+                assert cmp.compare(previousFt, frameTuple) < 0;
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrameFactory.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrameFactory.java
index 9508df5..b445fa8 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrameFactory.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrameFactory.java
@@ -22,8 +22,9 @@
 public class BTreeNSMLeafFrameFactory implements ITreeIndexFrameFactory {
 
     private static final long serialVersionUID = 1L;
+
     private final ITreeIndexTupleWriterFactory tupleWriterFactory;
-    
+
     public BTreeNSMLeafFrameFactory(ITreeIndexTupleWriterFactory tupleWriterFactory) {
         this.tupleWriterFactory = tupleWriterFactory;
     }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java
index 5f507f5..e51ee99 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java
@@ -23,8 +23,8 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
 public class OrderedSlotManager extends AbstractSlotManager {
-    
-	@Override
+
+    @Override
     public int findTupleIndex(ITupleReference searchKey, ITreeIndexTupleReference frameTuple, MultiComparator multiCmp,
             FindTupleMode mode, FindTupleNoExactMatchPolicy matchPolicy) {
         if (frame.getTupleCount() <= 0) {
@@ -34,12 +34,12 @@
         int mid;
         int begin = 0;
         int end = frame.getTupleCount() - 1;
-        
+
         while (begin <= end) {
             mid = (begin + end) / 2;
-            frameTuple.resetByTupleIndex(frame, mid);            
-            
-            int cmp = multiCmp.compare(searchKey, frameTuple);            
+            frameTuple.resetByTupleIndex(frame, mid);
+
+            int cmp = multiCmp.compare(searchKey, frameTuple);
             if (cmp < 0) {
                 end = mid - 1;
             } else if (cmp > 0) {
@@ -87,7 +87,7 @@
             }
         }
     }
-    
+
     @Override
     public int insertSlot(int tupleIndex, int tupleOff) {
         int slotOff = getSlotOff(tupleIndex);
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 89e8870..86bc32a 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
@@ -16,102 +16,76 @@
 package edu.uci.ics.hyracks.storage.am.btree.impls;
 
 import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.api.ITupleAcceptor;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeNonExistentKeyException;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeNotUpdateableException;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext.PageValidationInfo;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
 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.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import edu.uci.ics.hyracks.storage.am.common.api.IndexType;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+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.api.UnsortedInputException;
 import edu.uci.ics.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.impls.AbstractTreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NodeFrontier;
+import edu.uci.ics.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexUtils;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
-public class BTree implements ITreeIndex {
+public class BTree extends AbstractTreeIndex {
 
     public static final float DEFAULT_FILL_FACTOR = 0.7f;
 
     private final static long RESTART_OP = Long.MIN_VALUE;
+    private final static long FULL_RESTART_OP = Long.MIN_VALUE + 1;
     private final static int MAX_RESTARTS = 10;
-    private final static int rootPage = 1;
-        
-    private final IFreePageManager freePageManager;
-    private final IBufferCache bufferCache;    
-    private final IOperationCallback opCallback;
-    private final ITreeIndexFrameFactory interiorFrameFactory;
-    private final ITreeIndexFrameFactory leafFrameFactory;
-    private final int fieldCount;
-    private final IBinaryComparatorFactory[] cmpFactories;
+
+    private final AtomicInteger smoCounter;
     private final ReadWriteLock treeLatch;
-    private int fileId;
 
-    public BTree(IBufferCache bufferCache, IOperationCallback opCallback, int fieldCount, IBinaryComparatorFactory[] cmpFactories, IFreePageManager freePageManager,
-            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory) {
-        this.bufferCache = bufferCache;
-        this.opCallback = opCallback;
-        this.fieldCount = fieldCount;
-        this.cmpFactories = cmpFactories;
-        this.interiorFrameFactory = interiorFrameFactory;
-        this.leafFrameFactory = leafFrameFactory;        
-        this.freePageManager = freePageManager;
+    public BTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider, IFreePageManager freePageManager,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
+            IBinaryComparatorFactory[] cmpFactories, int fieldCount, FileReference file) {
+        super(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
+                fieldCount, file);
         this.treeLatch = new ReentrantReadWriteLock(true);
-    }
-
-    @Override
-    public void create(int fileId) throws HyracksDataException {
-        treeLatch.writeLock().lock();
-        try {
-            ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
-            ITreeIndexMetaDataFrame metaFrame = freePageManager.getMetaDataFrameFactory().createFrame();
-            this.fileId = fileId;
-            freePageManager.open(fileId);
-            freePageManager.init(metaFrame, rootPage);
-            initRoot(leafFrame, true);
-        } finally {
-            treeLatch.writeLock().unlock();
-        }
-    }
-
-    @Override
-    public void open(int fileId) {    	
-    	this.fileId = fileId;
-    	freePageManager.open(fileId);
-    }
-
-    @Override
-    public void close() {
-        fileId = -1;
-        freePageManager.close();
+        this.smoCounter = new AtomicInteger();
     }
 
     private void diskOrderScan(ITreeIndexCursor icursor, BTreeOpContext ctx) throws HyracksDataException {
-        TreeDiskOrderScanCursor cursor = (TreeDiskOrderScanCursor) icursor;
+        TreeIndexDiskOrderScanCursor cursor = (TreeIndexDiskOrderScanCursor) icursor;
         ctx.reset();
         RangePredicate diskOrderScanPred = new RangePredicate(null, null, true, true, ctx.cmp, ctx.cmp);
         int currentPageId = rootPage;
@@ -124,6 +98,8 @@
             cursor.setCurrentPageId(currentPageId);
             cursor.setMaxPageId(maxPageId);
             ctx.cursorInitialState.setPage(page);
+            ctx.cursorInitialState.setSearchOperationCallback(ctx.searchCallback);
+            ctx.cursorInitialState.setOriginialKeyComparator(ctx.cmp);
             cursor.open(ctx.cursorInitialState, diskOrderScanPred);
         } catch (Exception e) {
             page.releaseReadLatch();
@@ -132,6 +108,69 @@
         }
     }
 
+    public void validate() throws HyracksDataException {
+        // Stack validation protocol:
+        //      * parent pushes the validation information onto the stack before validation
+        //      * child pops the validation information off of the stack after validating
+        BTreeAccessor accessor = (BTreeAccessor) createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+        PageValidationInfo pvi = accessor.ctx.createPageValidationInfo(null);
+        accessor.ctx.validationInfos.addFirst(pvi);
+        validate(accessor.ctx, rootPage);
+    }
+
+    private void validate(BTreeOpContext ctx, int pageId) throws HyracksDataException {
+        ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+        ctx.interiorFrame.setPage(page);
+        PageValidationInfo currentPvi = ctx.validationInfos.peekFirst();
+
+        boolean isLeaf = ctx.interiorFrame.isLeaf();
+        if (isLeaf) {
+            ctx.leafFrame.setPage(page);
+            ctx.leafFrame.validate(currentPvi);
+        } else {
+            PageValidationInfo nextPvi = ctx.createPageValidationInfo(currentPvi);
+            List<Integer> children = ((BTreeNSMInteriorFrame) ctx.interiorFrame).getChildren(ctx.cmp);
+            ctx.interiorFrame.validate(currentPvi);
+            for (int i = 0; i < children.size(); i++) {
+                ctx.interiorFrame.setPage(page);
+
+                if (children.size() == 1) {
+                    // There is a single child pointer with no keys, so propagate both low and high ranges
+                    nextPvi.propagateLowRangeKey(currentPvi);
+                    nextPvi.propagateHighRangeKey(currentPvi);
+                } else if (i == 0) {
+                    // There is more than one child pointer and this is the left-most child pointer, so:
+                    //      1) propagate the low range key from the parent
+                    //      2) adjust the high range key
+                    nextPvi.propagateLowRangeKey(currentPvi);
+                    ctx.interiorFrameTuple.resetByTupleIndex(ctx.interiorFrame, i);
+                    nextPvi.adjustHighRangeKey(ctx.interiorFrameTuple);
+                } else if (i == children.size() - 1) {
+                    // There is more than one child pointer and this is the right-most child pointer, so:
+                    //      1) propagate the high range key from the parent
+                    //      2) adjust the low range key
+                    nextPvi.propagateHighRangeKey(currentPvi);
+                    ctx.interiorFrameTuple.resetByTupleIndex(ctx.interiorFrame, i - 1);
+                    nextPvi.adjustLowRangeKey(ctx.interiorFrameTuple);
+                } else {
+                    // There is more than one child pointer and this pointer is not the left/right-most pointer, so:
+                    //      1) adjust the low range key
+                    //      2) adjust the high range key
+                    ctx.interiorFrameTuple.resetByTupleIndex(ctx.interiorFrame, i - 1);
+                    nextPvi.adjustLowRangeKey(ctx.interiorFrameTuple);
+                    ctx.interiorFrameTuple.resetByTupleIndex(ctx.interiorFrame, i);
+                    nextPvi.adjustHighRangeKey(ctx.interiorFrameTuple);
+                }
+
+                ctx.validationInfos.addFirst(nextPvi);
+                validate(ctx, children.get(i));
+            }
+        }
+        bufferCache.unpin(page);
+        ctx.validationInfos.removeFirst();
+    }
+
     private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
             throws TreeIndexException, HyracksDataException {
         ctx.reset();
@@ -176,24 +215,17 @@
             }
         }
         if (ctx.smPages.size() > 0) {
+            if (ctx.smoCount == Integer.MAX_VALUE) {
+                smoCounter.set(0);
+            } else {
+                smoCounter.incrementAndGet();
+            }
             treeLatch.writeLock().unlock();
             ctx.smPages.clear();
         }
         ctx.interiorFrame.setPage(originalPage);
     }
 
-    private void initRoot(ITreeIndexFrame leafFrame, boolean firstInit) throws HyracksDataException {
-        ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), firstInit);
-        rootNode.acquireWriteLatch();
-        try {
-            leafFrame.setPage(rootNode);
-            leafFrame.initBuffer((byte) 0);
-        } finally {
-            rootNode.releaseWriteLatch();
-            bufferCache.unpin(rootNode);
-        }
-    }
-    
     private void createNewRoot(BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
         // Make sure the root is always in the same page.
         ICachedPage leftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, ctx.splitKey.getLeftPage()),
@@ -209,9 +241,13 @@
                         .getBuffer().capacity());
                 ctx.interiorFrame.setPage(newLeftNode);
                 ctx.interiorFrame.setSmFlag(false);
+                // Remember LSN to set it in the root.
+                long leftNodeLSN = ctx.interiorFrame.getPageLsn();
                 // Initialize new root (leftNode becomes new root).
                 ctx.interiorFrame.setPage(leftNode);
                 ctx.interiorFrame.initBuffer((byte) (ctx.interiorFrame.getLevel() + 1));
+                // Copy over LSN.
+                ctx.interiorFrame.setPageLsn(leftNodeLSN);
                 // Will be cleared later in unsetSmPages.
                 ctx.interiorFrame.setSmFlag(true);
                 ctx.splitKey.setLeftPage(newLeftId);
@@ -226,8 +262,9 @@
             bufferCache.unpin(leftNode);
         }
     }
-    
-    private void insertUpdateOrDelete(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+
+    private void insertUpdateOrDelete(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException,
+            TreeIndexException {
         ctx.reset();
         ctx.pred.setLowKeyComparator(ctx.cmp);
         ctx.pred.setHighKeyComparator(ctx.cmp);
@@ -239,11 +276,18 @@
         // due to ongoing structure modifications during the descent.
         boolean repeatOp = true;
         while (repeatOp && ctx.opRestarts < MAX_RESTARTS) {
+            ctx.smoCount = smoCounter.get();
             performOp(rootPage, null, true, ctx);
             // Do we need to restart from the (possibly new) root?
-            if (!ctx.pageLsns.isEmpty() && ctx.pageLsns.getLast() == RESTART_OP) {
-                ctx.pageLsns.removeLast(); // pop the restart op indicator
-                continue;
+            if (!ctx.pageLsns.isEmpty()) {
+                if (ctx.pageLsns.getLast() == FULL_RESTART_OP) {
+                    ctx.pageLsns.clear();
+                    continue;
+                } else if (ctx.pageLsns.getLast() == RESTART_OP) {
+                    ctx.pageLsns.removeLast(); // pop the restart op indicator
+                    continue;
+                }
+
             }
             // Split key propagated?
             if (ctx.splitKey.getBuffer() != null) {
@@ -253,13 +297,19 @@
             unsetSmPages(ctx);
             repeatOp = false;
         }
+
+        if (ctx.opRestarts >= MAX_RESTARTS) {
+            throw new BTreeException("Operation exceeded the maximum number of restarts");
+        }
     }
-    
+
     private void insert(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+        ctx.modificationCallback.before(tuple);
         insertUpdateOrDelete(tuple, ctx);
     }
-    
+
     private void upsert(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+        ctx.modificationCallback.before(tuple);
         insertUpdateOrDelete(tuple, ctx);
     }
 
@@ -270,18 +320,22 @@
         if (fieldCount == ctx.cmp.getKeyFieldCount()) {
             throw new BTreeNotUpdateableException("Cannot perform updates when the entire tuple forms the key.");
         }
+        ctx.modificationCallback.before(tuple);
         insertUpdateOrDelete(tuple, ctx);
     }
-    
+
     private void delete(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+        ctx.modificationCallback.before(tuple);
         insertUpdateOrDelete(tuple, ctx);
     }
-    
-    private boolean insertLeaf(ITupleReference tuple, int targetTupleIndex, int pageId, BTreeOpContext ctx) throws Exception {
+
+    private boolean insertLeaf(ITupleReference tuple, int targetTupleIndex, int pageId, BTreeOpContext ctx)
+            throws Exception {
         boolean restartOp = false;
         FrameOpSpaceStatus spaceStatus = ctx.leafFrame.hasSpaceInsert(tuple);
         switch (spaceStatus) {
             case SUFFICIENT_CONTIGUOUS_SPACE: {
+                ctx.modificationCallback.found(null, tuple);
                 ctx.leafFrame.insert(tuple, targetTupleIndex);
                 ctx.splitKey.reset();
                 break;
@@ -291,11 +345,12 @@
                 if (slotsChanged) {
                     targetTupleIndex = ctx.leafFrame.findInsertTupleIndex(tuple);
                 }
+                ctx.modificationCallback.found(null, tuple);
                 ctx.leafFrame.insert(tuple, targetTupleIndex);
                 ctx.splitKey.reset();
                 break;
             }
-            case INSUFFICIENT_SPACE: {            	
+            case INSUFFICIENT_SPACE: {
                 // Try compressing the page first and see if there is space available.
                 boolean reCompressed = ctx.leafFrame.compress();
                 if (reCompressed) {
@@ -304,18 +359,20 @@
                     spaceStatus = ctx.leafFrame.hasSpaceInsert(tuple);
                 }
                 if (spaceStatus == FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE) {
+                    ctx.modificationCallback.found(null, tuple);
                     ctx.leafFrame.insert(tuple, targetTupleIndex);
                     ctx.splitKey.reset();
                 } else {
-                	restartOp = performLeafSplit(pageId, tuple, ctx);
+                    restartOp = performLeafSplit(pageId, tuple, ctx, -1);
                 }
                 break;
             }
-        }        
+        }
         return restartOp;
     }
-    
-    private boolean performLeafSplit(int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {    	
+
+    private boolean performLeafSplit(int pageId, ITupleReference tuple, BTreeOpContext ctx, int updateTupleIndex)
+            throws Exception {
         // We must never hold a latch on a page while waiting to obtain the tree
         // latch, because it this could lead to a latch-deadlock.
         // If we can't get the tree latch, we return, release our page latches,
@@ -323,16 +380,30 @@
         // Lock is released in unsetSmPages(), after sm has fully completed.
         if (!treeLatch.writeLock().tryLock()) {
             return true;
+        } else {
+            int tempSmoCount = smoCounter.get();
+            if (tempSmoCount != ctx.smoCount) {
+                treeLatch.writeLock().unlock();
+                return true;
+            }
         }
         int rightPageId = freePageManager.getFreePage(ctx.metaFrame);
-        ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId),
-                true);
+        ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId), true);
         rightNode.acquireWriteLatch();
         try {
             IBTreeLeafFrame rightFrame = ctx.createLeafFrame();
             rightFrame.setPage(rightNode);
             rightFrame.initBuffer((byte) 0);
             rightFrame.setMultiComparator(ctx.cmp);
+
+            // Perform an update (delete + insert) if the updateTupleIndex != -1
+            if (updateTupleIndex != -1) {
+                ITupleReference beforeTuple = ctx.leafFrame.getMatchingKeyTuple(tuple, updateTupleIndex);
+                ctx.modificationCallback.found(beforeTuple, tuple);
+                ctx.leafFrame.delete(tuple, updateTupleIndex);
+            } else {
+                ctx.modificationCallback.found(null, tuple);
+            }
             ctx.leafFrame.split(rightFrame, tuple, ctx.splitKey);
 
             ctx.smPages.add(pageId);
@@ -343,10 +414,6 @@
             rightFrame.setNextLeaf(ctx.leafFrame.getNextLeaf());
             ctx.leafFrame.setNextLeaf(rightPageId);
 
-            // TODO: we just use increasing numbers as pageLsn,
-            // we
-            // should tie this together with the LogManager and
-            // TransactionManager
             rightFrame.setPageLsn(rightFrame.getPageLsn() + 1);
             ctx.leafFrame.setPageLsn(ctx.leafFrame.getPageLsn() + 1);
 
@@ -360,63 +427,60 @@
         }
         return false;
     }
-    
-    private boolean updateLeaf(ITupleReference tuple, int oldTupleIndex, int pageId, BTreeOpContext ctx) throws Exception {
+
+    private boolean updateLeaf(ITupleReference tuple, int oldTupleIndex, int pageId, BTreeOpContext ctx)
+            throws Exception {
         FrameOpSpaceStatus spaceStatus = ctx.leafFrame.hasSpaceUpdate(tuple, oldTupleIndex);
+        ITupleReference beforeTuple = ctx.leafFrame.getMatchingKeyTuple(tuple, oldTupleIndex);
         boolean restartOp = false;
         switch (spaceStatus) {
             case SUFFICIENT_INPLACE_SPACE: {
+                ctx.modificationCallback.found(beforeTuple, tuple);
                 ctx.leafFrame.update(tuple, oldTupleIndex, true);
                 ctx.splitKey.reset();
                 break;
             }
             case SUFFICIENT_CONTIGUOUS_SPACE: {
+                ctx.modificationCallback.found(beforeTuple, tuple);
                 ctx.leafFrame.update(tuple, oldTupleIndex, false);
                 ctx.splitKey.reset();
                 break;
-            }                
+            }
             case SUFFICIENT_SPACE: {
                 // Delete the old tuple, compact the frame, and insert the new tuple.
+                ctx.modificationCallback.found(beforeTuple, tuple);
                 ctx.leafFrame.delete(tuple, oldTupleIndex);
                 ctx.leafFrame.compact();
                 int targetTupleIndex = ctx.leafFrame.findInsertTupleIndex(tuple);
                 ctx.leafFrame.insert(tuple, targetTupleIndex);
                 ctx.splitKey.reset();
                 break;
-            }                
+            }
             case INSUFFICIENT_SPACE: {
-                // Delete the old tuple, and try compressing the page to make space available.
-                ctx.leafFrame.delete(tuple, oldTupleIndex);
-                ctx.leafFrame.compress();
-                // We need to insert the new tuple, so check if there is space.
-                spaceStatus = ctx.leafFrame.hasSpaceInsert(tuple);                
-                if (spaceStatus == FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE) {
-                    int targetTupleIndex = ctx.leafFrame.findInsertTupleIndex(tuple);
-                    ctx.leafFrame.insert(tuple, targetTupleIndex);
-                    ctx.splitKey.reset();
-                } else {
-                    restartOp = performLeafSplit(pageId, tuple, ctx);
-                }
+                restartOp = performLeafSplit(pageId, tuple, ctx, oldTupleIndex);
                 break;
             }
         }
         return restartOp;
     }
 
-    private boolean upsertLeaf(ITupleReference tuple, int targetTupleIndex, int pageId, BTreeOpContext ctx) throws Exception {
+    private boolean upsertLeaf(ITupleReference tuple, int targetTupleIndex, int pageId, BTreeOpContext ctx)
+            throws Exception {
         boolean restartOp = false;
-        ITupleReference beforeTuple = ctx.leafFrame.getUpsertBeforeTuple(tuple, targetTupleIndex);
-        if (beforeTuple == null) {
-            opCallback.pre(null);
-            restartOp = insertLeaf(tuple, targetTupleIndex, pageId, ctx);
+        ITupleReference beforeTuple = ctx.leafFrame.getMatchingKeyTuple(tuple, targetTupleIndex);
+        if (ctx.acceptor.accept(beforeTuple)) {
+            if (beforeTuple == null) {
+                restartOp = insertLeaf(tuple, targetTupleIndex, pageId, ctx);
+            } else {
+                restartOp = updateLeaf(tuple, targetTupleIndex, pageId, ctx);
+            }
         } else {
-            opCallback.pre(beforeTuple);
-            restartOp = updateLeaf(tuple, targetTupleIndex, pageId, ctx);
+            targetTupleIndex = ctx.leafFrame.findInsertTupleIndex(tuple);
+            restartOp = insertLeaf(tuple, targetTupleIndex, pageId, ctx);
         }
-        opCallback.post(tuple);
         return restartOp;
     }
-    
+
     private void insertInterior(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx)
             throws Exception {
         ctx.interiorFrame.setPage(node);
@@ -439,9 +503,6 @@
                     ctx.smPages.add(rightPageId);
                     ctx.interiorFrame.setSmFlag(true);
                     rightFrame.setSmFlag(true);
-                    // TODO: we just use increasing numbers as pageLsn, we
-                    // should tie this together with the LogManager and
-                    // TransactionManager
                     rightFrame.setPageLsn(rightFrame.getPageLsn() + 1);
                     ctx.interiorFrame.setPageLsn(ctx.interiorFrame.getPageLsn() + 1);
 
@@ -451,7 +512,7 @@
                     bufferCache.unpin(rightNode);
                 }
                 break;
-            }                
+            }
 
             case SUFFICIENT_CONTIGUOUS_SPACE: {
                 ctx.interiorFrame.insert(tuple, targetTupleIndex);
@@ -471,7 +532,8 @@
         }
     }
 
-    private boolean deleteLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
+    private boolean deleteLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx)
+            throws Exception {
         // Simply delete the tuple, and don't do any rebalancing.
         // This means that there could be underflow, even an empty page that is
         // pointed to by an interior node.
@@ -479,12 +541,14 @@
             throw new BTreeNonExistentKeyException("Trying to delete a tuple with a nonexistent key in leaf node.");
         }
         int tupleIndex = ctx.leafFrame.findDeleteTupleIndex(tuple);
+        ITupleReference beforeTuple = ctx.leafFrame.getMatchingKeyTuple(tuple, tupleIndex);
+        ctx.modificationCallback.found(beforeTuple, tuple);
         ctx.leafFrame.delete(tuple, tupleIndex);
         return false;
     }
 
     private final boolean acquireLatch(ICachedPage node, BTreeOpContext ctx, boolean isLeaf) {
-        if (!isLeaf || (ctx.op == IndexOp.SEARCH && !ctx.cursor.exclusiveLatchNodes())) {
+        if (!isLeaf || (ctx.op == IndexOperation.SEARCH && !ctx.cursor.exclusiveLatchNodes())) {
             node.acquireReadLatch();
             return true;
         } else {
@@ -493,24 +557,23 @@
         }
     }
 
-    private boolean isConsistent(int pageId, BTreeOpContext ctx) throws Exception {
+    private ICachedPage isConsistent(int pageId, BTreeOpContext ctx) throws Exception {
         ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
         node.acquireReadLatch();
         ctx.interiorFrame.setPage(node);
-        boolean isConsistent = false;
-        try {
-            isConsistent = ctx.pageLsns.getLast() == ctx.interiorFrame.getPageLsn();
-        } finally {
+        boolean isConsistent = ctx.pageLsns.getLast() == ctx.interiorFrame.getPageLsn();
+        if (!isConsistent) {
             node.releaseReadLatch();
             bufferCache.unpin(node);
+            return null;
         }
-        return isConsistent;
+        return node;
     }
 
-    private void performOp(int pageId, ICachedPage parent, boolean parentIsReadLatched, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+    private void performOp(int pageId, ICachedPage parent, boolean parentIsReadLatched, BTreeOpContext ctx)
+            throws HyracksDataException, TreeIndexException {
         ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
         ctx.interiorFrame.setPage(node);
-        
         // this check performs an unprotected read in the page
         // the following could happen: TODO fill out
         boolean unsafeIsLeaf = ctx.interiorFrame.isLeaf();
@@ -526,9 +589,9 @@
             // Latch coupling: unlatch parent.
             if (parent != null) {
                 if (parentIsReadLatched) {
-                	parent.releaseReadLatch();
+                    parent.releaseReadLatch();
                 } else {
-                	parent.releaseWriteLatch();
+                    parent.releaseWriteLatch();
                 }
                 bufferCache.unpin(parent);
             }
@@ -542,39 +605,41 @@
                         int childPageId = ctx.interiorFrame.getChildPageId(ctx.pred);
                         performOp(childPageId, node, isReadLatched, ctx);
 
-                        if (!ctx.pageLsns.isEmpty() && ctx.pageLsns.getLast() == RESTART_OP) {
-                            // Pop the restart op indicator.
-                            ctx.pageLsns.removeLast();                            
-                            if (isConsistent(pageId, ctx)) {
-                                // Pin and latch page again, since it was unpinned and unlatched in call to performOp (passed as parent).
-                                node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
-                                node.acquireReadLatch();
-                                ctx.interiorFrame.setPage(node);
-                                isReadLatched = true;
-                                // Descend the tree again.                                
-                                continue;
-                            } else {
-                                // Pop pageLsn of this page (version seen by this op during descent).
-                                ctx.pageLsns.removeLast(); 
-                                // This node is not consistent set the restart indicator for upper level.
-                                ctx.pageLsns.add(RESTART_OP);
+                        if (!ctx.pageLsns.isEmpty()) {
+                            if (ctx.pageLsns.getLast() == FULL_RESTART_OP) {
                                 break;
+                            } else if (ctx.pageLsns.getLast() == RESTART_OP) {
+                                // Pop the restart op indicator.
+                                ctx.pageLsns.removeLast();
+                                node = isConsistent(pageId, ctx);
+                                if (node != null) {
+                                    isReadLatched = true;
+                                    // Descend the tree again.                                
+                                    continue;
+                                } else {
+                                    // Pop pageLsn of this page (version seen by this op during descent).
+                                    ctx.pageLsns.removeLast();
+                                    // This node is not consistent set the restart indicator for upper level.
+                                    ctx.pageLsns.add(RESTART_OP);
+                                    break;
+                                }
                             }
                         }
-                        
+
                         switch (ctx.op) {
                             case INSERT:
                             case UPSERT:
                             case UPDATE: {
                                 // Is there a propagated split key?
                                 if (ctx.splitKey.getBuffer() != null) {
-                                    ICachedPage interiorNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+                                    ICachedPage interiorNode = bufferCache.pin(
+                                            BufferedFileHandle.getDiskPageId(fileId, pageId), false);
                                     interiorNode.acquireWriteLatch();
                                     try {
-                                        // Insert or update op. Both can cause split keys to propagate upwards.                                            
+                                        // Insert or update op. Both can cause split keys to propagate upwards. 
                                         insertInterior(interiorNode, pageId, ctx.splitKey.getTuple(), ctx);
                                     } finally {
-                                    	interiorNode.releaseWriteLatch();
+                                        interiorNode.releaseWriteLatch();
                                         bufferCache.unpin(interiorNode);
                                     }
                                 } else {
@@ -582,14 +647,15 @@
                                 }
                                 break;
                             }
-                            
+
                             case DELETE: {
                                 if (ctx.splitKey.getBuffer() != null) {
-                                    throw new BTreeException("Split key was propagated during delete. Delete allows empty leaf pages.");
+                                    throw new BTreeException(
+                                            "Split key was propagated during delete. Delete allows empty leaf pages.");
                                 }
                                 break;
                             }
-                                
+
                             default: {
                                 // Do nothing for Search and DiskOrderScan.
                                 break;
@@ -601,9 +667,9 @@
                 } else { // smFlag
                     ctx.opRestarts++;
                     if (isReadLatched) {
-                    	node.releaseReadLatch();
+                        node.releaseReadLatch();
                     } else {
-                    	node.releaseWriteLatch();
+                        node.releaseWriteLatch();
                     }
                     bufferCache.unpin(node);
 
@@ -612,9 +678,9 @@
                     // instead we just immediately release the lock. this is
                     // inefficient but still correct and will not cause
                     // latch-deadlock
-                    treeLatch.writeLock().lock();
-                    treeLatch.writeLock().unlock();
-                    
+                    treeLatch.readLock().lock();
+                    treeLatch.readLock().unlock();
+
                     // unwind recursion and restart operation, find lowest page
                     // with a pageLsn as seen by this operation during descent
                     ctx.pageLsns.removeLast(); // pop current page lsn
@@ -624,10 +690,10 @@
                 }
             } else { // isLeaf and !smFlag
                 // We may have to restart an op to avoid latch deadlock.
-            	boolean restartOp = false;
-            	ctx.leafFrame.setPage(node);
-            	switch (ctx.op) {
-                    case INSERT: {                        
+                boolean restartOp = false;
+                ctx.leafFrame.setPage(node);
+                switch (ctx.op) {
+                    case INSERT: {
                         int targetTupleIndex = ctx.leafFrame.findInsertTupleIndex(ctx.pred.getLowKey());
                         restartOp = insertLeaf(ctx.pred.getLowKey(), targetTupleIndex, pageId, ctx);
                         break;
@@ -639,323 +705,72 @@
                     }
                     case UPDATE: {
                         int oldTupleIndex = ctx.leafFrame.findUpdateTupleIndex(ctx.pred.getLowKey());
-                    	restartOp = updateLeaf(ctx.pred.getLowKey(), oldTupleIndex, pageId, ctx);
+                        restartOp = updateLeaf(ctx.pred.getLowKey(), oldTupleIndex, pageId, ctx);
                         break;
                     }
                     case DELETE: {
-                    	restartOp = deleteLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
+                        restartOp = deleteLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
                         break;
                     }
                     case SEARCH: {
+                        ctx.cursorInitialState.setSearchOperationCallback(ctx.searchCallback);
+                        ctx.cursorInitialState.setOriginialKeyComparator(ctx.cmp);
                         ctx.cursorInitialState.setPage(node);
+                        ctx.cursorInitialState.setPageId(pageId);
                         ctx.cursor.open(ctx.cursorInitialState, ctx.pred);
                         break;
                     }
                 }
-            	if (ctx.op != IndexOp.SEARCH) {
-            	    node.releaseWriteLatch();
+                if (ctx.op != IndexOperation.SEARCH) {
+                    node.releaseWriteLatch();
                     bufferCache.unpin(node);
-            	}
-            	if (restartOp) {
-            		ctx.pageLsns.removeLast();
-                    ctx.pageLsns.add(RESTART_OP);
-            	}
+                }
+                if (restartOp) {
+                    // Wait for the SMO to finish before restarting.
+                    treeLatch.readLock().lock();
+                    treeLatch.readLock().unlock();
+                    ctx.pageLsns.removeLast();
+                    ctx.pageLsns.add(FULL_RESTART_OP);
+                }
             }
         } catch (TreeIndexException e) {
-        	if (!ctx.exceptionHandled) {
-        		if (node != null) {
-        			if (isReadLatched) {
-        				node.releaseReadLatch();
-        			} else {
-        				node.releaseWriteLatch();
-        			}
-        			bufferCache.unpin(node);
-        			ctx.exceptionHandled = true;
-        		}
+            if (!ctx.exceptionHandled) {
+                if (node != null) {
+                    if (isReadLatched) {
+                        node.releaseReadLatch();
+                    } else {
+                        node.releaseWriteLatch();
+                    }
+                    bufferCache.unpin(node);
+                    ctx.exceptionHandled = true;
+                }
             }
             throw e;
         } catch (Exception e) {
-        	e.printStackTrace();
-        	if (node != null) {
-        		if (isReadLatched) {
-    				node.releaseReadLatch();
-    			} else {
-    				node.releaseWriteLatch();
-    			}
-        		bufferCache.unpin(node);
-        	}
+            e.printStackTrace();
+            if (node != null) {
+                if (isReadLatched) {
+                    node.releaseReadLatch();
+                } else {
+                    node.releaseWriteLatch();
+                }
+                bufferCache.unpin(node);
+            }
             BTreeException wrappedException = new BTreeException(e);
             ctx.exceptionHandled = true;
             throw wrappedException;
         }
     }
 
-    public class BulkLoadContext implements IIndexBulkLoadContext {
-        public final MultiComparator cmp;
-        public final int slotSize;
-        public final int leafMaxBytes;
-        public final int interiorMaxBytes;
-        public final BTreeSplitKey splitKey;
-        // we maintain a frontier of nodes for each level
-        private final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<NodeFrontier>();
-        private final IBTreeLeafFrame leafFrame;
-        private final IBTreeInteriorFrame interiorFrame;
-        private final ITreeIndexMetaDataFrame metaFrame;
-        private final ITreeIndexTupleWriter tupleWriter;        
-        
-        public BulkLoadContext(float fillFactor, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
-                ITreeIndexMetaDataFrame metaFrame, IBinaryComparatorFactory[] cmpFactories) throws HyracksDataException {
-            this.cmp = MultiComparator.create(cmpFactories);
-            
-        	leafFrame.setMultiComparator(cmp);
-        	interiorFrame.setMultiComparator(cmp);
-        	
-            splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
-            tupleWriter = leafFrame.getTupleWriter();
-
-            NodeFrontier leafFrontier = new NodeFrontier(leafFrame.createTupleReference());
-            leafFrontier.pageId = freePageManager.getFreePage(metaFrame);
-            leafFrontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId),
-                    true);
-            leafFrontier.page.acquireWriteLatch();
-
-            interiorFrame.setPage(leafFrontier.page);
-            interiorFrame.initBuffer((byte) 0);
-            interiorMaxBytes = (int) ((float) interiorFrame.getBuffer().capacity() * fillFactor);
-
-            leafFrame.setPage(leafFrontier.page);
-            leafFrame.initBuffer((byte) 0);
-            leafMaxBytes = (int) ((float) leafFrame.getBuffer().capacity() * fillFactor);
-
-            slotSize = leafFrame.getSlotSize();
-
-            this.leafFrame = leafFrame;
-            this.interiorFrame = interiorFrame;
-            this.metaFrame = metaFrame;
-
-            nodeFrontiers.add(leafFrontier);
-        }
-
-        private void addLevel() throws HyracksDataException {
-            NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
-            frontier.pageId = freePageManager.getFreePage(metaFrame);
-            frontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, frontier.pageId), true);
-            frontier.page.acquireWriteLatch();
-            frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());
-            interiorFrame.setPage(frontier.page);
-            interiorFrame.initBuffer((byte) nodeFrontiers.size());
-            nodeFrontiers.add(frontier);
-        }
+    private BTreeOpContext createOpContext(IIndexAccessor accessor,
+            IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback) {
+        return new BTreeOpContext(accessor, leafFrameFactory, interiorFrameFactory, freePageManager
+                .getMetaDataFrameFactory().createFrame(), cmpFactories, modificationCallback, searchCallback);
     }
 
-    private void propagateBulk(BulkLoadContext ctx, int level) throws HyracksDataException {
-
-        if (ctx.splitKey.getBuffer() == null)
-            return;
-
-        if (level >= ctx.nodeFrontiers.size())
-            ctx.addLevel();
-
-        NodeFrontier frontier = ctx.nodeFrontiers.get(level);
-        ctx.interiorFrame.setPage(frontier.page);
-
-        ITupleReference tuple = ctx.splitKey.getTuple();
-        int spaceNeeded = ctx.tupleWriter.bytesRequired(tuple, 0, ctx.cmp.getKeyFieldCount()) + ctx.slotSize + 4;
-        int spaceUsed = ctx.interiorFrame.getBuffer().capacity() - ctx.interiorFrame.getTotalFreeSpace();
-        if (spaceUsed + spaceNeeded > ctx.interiorMaxBytes) {
-
-            BTreeSplitKey copyKey = ctx.splitKey.duplicate(ctx.leafFrame.getTupleWriter().createTupleReference());
-            tuple = copyKey.getTuple();
-
-            frontier.lastTuple.resetByTupleIndex(ctx.interiorFrame, ctx.interiorFrame.getTupleCount() - 1);
-            int splitKeySize = ctx.tupleWriter.bytesRequired(frontier.lastTuple, 0, ctx.cmp.getKeyFieldCount());
-            ctx.splitKey.initData(splitKeySize);
-            ctx.tupleWriter
-                    .writeTupleFields(frontier.lastTuple, 0, ctx.cmp.getKeyFieldCount(), ctx.splitKey.getBuffer().array(), 0);
-            ctx.splitKey.getTuple().resetByTupleOffset(ctx.splitKey.getBuffer(), 0);
-            ctx.splitKey.setLeftPage(frontier.pageId);
-
-            ctx.interiorFrame.deleteGreatest();
-
-            frontier.page.releaseWriteLatch();
-            bufferCache.unpin(frontier.page);
-            frontier.pageId = freePageManager.getFreePage(ctx.metaFrame);
-
-            ctx.splitKey.setRightPage(frontier.pageId);
-            propagateBulk(ctx, level + 1);
-
-            frontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, frontier.pageId), true);
-            frontier.page.acquireWriteLatch();
-            ctx.interiorFrame.setPage(frontier.page);
-            ctx.interiorFrame.initBuffer((byte) level);
-        }
-        ctx.interiorFrame.insertSorted(tuple);
-    }
-
-    // assumes btree has been created and opened
-    @Override
-    public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws TreeIndexException, HyracksDataException {
-        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
-    	if (!isEmptyTree(leafFrame)) {
-    		throw new BTreeException("Trying to Bulk-load a non-empty BTree.");
-    	}
-    	
-        BulkLoadContext ctx = new BulkLoadContext(fillFactor, leafFrame,
-                (IBTreeInteriorFrame)interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory().createFrame(), cmpFactories);
-        ctx.splitKey.getTuple().setFieldCount(ctx.cmp.getKeyFieldCount());
-        return ctx;
-    }
-
-    @Override
-    public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException {
-        BulkLoadContext ctx = (BulkLoadContext) ictx;
-        NodeFrontier leafFrontier = ctx.nodeFrontiers.get(0);
-        IBTreeLeafFrame leafFrame = ctx.leafFrame;
-
-        int spaceNeeded = ctx.tupleWriter.bytesRequired(tuple) + ctx.slotSize;
-        int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-
-        // try to free space by compression
-        if (spaceUsed + spaceNeeded > ctx.leafMaxBytes) {
-            leafFrame.compress();
-            spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
-        }
-
-        if (spaceUsed + spaceNeeded > ctx.leafMaxBytes) {
-            leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
-            int splitKeySize = ctx.tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, ctx.cmp.getKeyFieldCount());
-            ctx.splitKey.initData(splitKeySize);
-            ctx.tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, ctx.cmp.getKeyFieldCount(),
-                    ctx.splitKey.getBuffer().array(), 0);
-            ctx.splitKey.getTuple().resetByTupleOffset(ctx.splitKey.getBuffer(), 0);
-            ctx.splitKey.setLeftPage(leafFrontier.pageId);
-            leafFrontier.pageId = freePageManager.getFreePage(ctx.metaFrame);
-
-            leafFrame.setNextLeaf(leafFrontier.pageId);
-            leafFrontier.page.releaseWriteLatch();
-            bufferCache.unpin(leafFrontier.page);
-
-            ctx.splitKey.setRightPage(leafFrontier.pageId);
-            propagateBulk(ctx, 1);
-
-            leafFrontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId),
-                    true);
-            leafFrontier.page.acquireWriteLatch();
-            leafFrame.setPage(leafFrontier.page);
-            leafFrame.initBuffer((byte) 0);
-        }
-
-        leafFrame.setPage(leafFrontier.page);
-        leafFrame.insertSorted(tuple);
-    }
-
-    @Override
-    public void endBulkLoad(IIndexBulkLoadContext ictx) throws HyracksDataException {
-        // copy root
-        BulkLoadContext ctx = (BulkLoadContext) ictx;
-        ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), true);
-        rootNode.acquireWriteLatch();
-        NodeFrontier lastNodeFrontier = ctx.nodeFrontiers.get(ctx.nodeFrontiers.size() - 1);
-        IBTreeInteriorFrame interiorFrame = ctx.interiorFrame;
-        try {
-            ICachedPage toBeRoot = lastNodeFrontier.page;
-            System.arraycopy(toBeRoot.getBuffer().array(), 0, rootNode.getBuffer().array(), 0, toBeRoot.getBuffer()
-                    .capacity());
-        } finally {
-            rootNode.releaseWriteLatch();
-            bufferCache.unpin(rootNode);
-
-            // register old root as free page
-            freePageManager.addFreePage(ctx.metaFrame, lastNodeFrontier.pageId);
-
-            // make old root a free page
-            interiorFrame.setPage(lastNodeFrontier.page);
-            interiorFrame.initBuffer(freePageManager.getFreePageLevelIndicator());
-
-            // cleanup
-            for (int i = 0; i < ctx.nodeFrontiers.size(); i++) {
-                ctx.nodeFrontiers.get(i).page.releaseWriteLatch();
-                bufferCache.unpin(ctx.nodeFrontiers.get(i).page);
-            }
-        }
-    }
-
-    private BTreeOpContext createOpContext() {
-        return new BTreeOpContext(leafFrameFactory, interiorFrameFactory, freePageManager.getMetaDataFrameFactory()
-                .createFrame(), cmpFactories);
-    }
-    
-    public ITreeIndexFrameFactory getInteriorFrameFactory() {
-        return interiorFrameFactory;
-    }
-
-    public ITreeIndexFrameFactory getLeafFrameFactory() {
-        return leafFrameFactory;
-    }
-
-    public IBinaryComparatorFactory[] getComparatorFactories() {
-        return cmpFactories;
-    }
-
-    public IFreePageManager getFreePageManager() {
-        return freePageManager;
-    }
-
-    public int getRootPageId() {
-        return rootPage;
-    }    
-
-    @Override
-    public int getFieldCount() {
-        return fieldCount;
-    }
-
-    @Override
-    public IndexType getIndexType() {
-        return IndexType.BTREE;
-    }
-    
-    @Override
-    public int getFileId() {
-    	return fileId;
-    }
-    
-    @Override
-    public IBufferCache getBufferCache() {
-        return bufferCache;
-    }
-    
-    public byte getTreeHeight(IBTreeLeafFrame leafFrame) throws HyracksDataException {
-        ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false);
-        rootNode.acquireReadLatch();
-        try {
-            leafFrame.setPage(rootNode);
-            return leafFrame.getLevel();
-        } finally {
-            rootNode.releaseReadLatch();
-            bufferCache.unpin(rootNode);
-        }
-    }
-    
-    public boolean isEmptyTree(IBTreeLeafFrame leafFrame) throws HyracksDataException {
-    	ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false);
-        rootNode.acquireReadLatch();
-        try {
-            leafFrame.setPage(rootNode);
-            if (leafFrame.getLevel() == 0 && leafFrame.getTupleCount() == 0) {
-            	return true;
-            } else {
-            	return false;
-            }
-        } finally {
-            rootNode.releaseReadLatch();
-            bufferCache.unpin(rootNode);
-        }
-    }
-    
-    @SuppressWarnings("rawtypes") 
-    public String printTree(IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame, ISerializerDeserializer[] keySerdes)
-            throws Exception {
+    @SuppressWarnings("rawtypes")
+    public String printTree(IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
+            ISerializerDeserializer[] keySerdes) throws Exception {
         MultiComparator cmp = MultiComparator.create(cmpFactories);
         byte treeHeight = getTreeHeight(leafFrame);
         StringBuilder strBuilder = new StringBuilder();
@@ -963,9 +778,10 @@
         return strBuilder.toString();
     }
 
-    @SuppressWarnings("rawtypes") 
+    @SuppressWarnings("rawtypes")
     public void printTree(int pageId, ICachedPage parent, boolean unpin, IBTreeLeafFrame leafFrame,
-            IBTreeInteriorFrame interiorFrame, byte treeHeight, ISerializerDeserializer[] keySerdes, StringBuilder strBuilder, MultiComparator cmp) throws Exception {
+            IBTreeInteriorFrame interiorFrame, byte treeHeight, ISerializerDeserializer[] keySerdes,
+            StringBuilder strBuilder, MultiComparator cmp) throws Exception {
         ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
         node.acquireReadLatch();
         try {
@@ -984,16 +800,17 @@
             String keyString;
             if (interiorFrame.isLeaf()) {
                 leafFrame.setPage(node);
-                keyString = TreeIndexUtils.printFrameTuples(leafFrame, keySerdes);
+                keyString = printLeafFrameTuples(leafFrame, keySerdes);
             } else {
-                keyString = TreeIndexUtils.printFrameTuples(interiorFrame, keySerdes);
+                keyString = printInteriorFrameTuples(interiorFrame, keySerdes);
             }
 
             strBuilder.append(keyString + "\n");
             if (!interiorFrame.isLeaf()) {
                 ArrayList<Integer> children = ((BTreeNSMInteriorFrame) (interiorFrame)).getChildren(cmp);
                 for (int i = 0; i < children.size(); i++) {
-                    printTree(children.get(i), node, i == children.size() - 1, leafFrame, interiorFrame, treeHeight, keySerdes, strBuilder, cmp);
+                    printTree(children.get(i), node, i == children.size() - 1, leafFrame, interiorFrame, treeHeight,
+                            keySerdes, strBuilder, cmp);
                 }
             } else {
                 node.releaseReadLatch();
@@ -1007,75 +824,259 @@
     }
 
     @Override
-    public ITreeIndexAccessor createAccessor() {
-        return new BTreeAccessor(this);
+    public ITreeIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new BTreeAccessor(this, modificationCallback, searchCallback);
     }
-    
-	// TODO: Class should be private. But currently we need to expose the
-	// setOpContext() API to the LSM Tree for it to work correctly.
+
+    // TODO: Class should be private. But currently we need to expose the
+    // setOpContext() API to the LSM Tree for it to work correctly.
     public class BTreeAccessor implements ITreeIndexAccessor {
         private BTree btree;
         private BTreeOpContext ctx;
-        
-        public BTreeAccessor(BTree btree) {
+
+        public BTreeAccessor(BTree btree, IModificationOperationCallback modificationCalback,
+                ISearchOperationCallback searchCallback) {
             this.btree = btree;
-            this.ctx = btree.createOpContext();
+            this.ctx = btree.createOpContext(this, modificationCalback, searchCallback);
         }
-        
+
         @Override
         public void insert(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
-            ctx.reset(IndexOp.INSERT);
+            ctx.setOperation(IndexOperation.INSERT);
             btree.insert(tuple, ctx);
         }
 
         @Override
         public void update(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
-            ctx.reset(IndexOp.UPDATE);
+            ctx.setOperation(IndexOperation.UPDATE);
             btree.update(tuple, ctx);
         }
 
         @Override
         public void delete(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
-            ctx.reset(IndexOp.DELETE);
+            ctx.setOperation(IndexOperation.DELETE);
             btree.delete(tuple, ctx);
         }
-        
+
         @Override
         public void upsert(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
-            ctx.reset(IndexOp.UPSERT);
+            upsertIfConditionElseInsert(tuple, UnconditionalTupleAcceptor.INSTANCE);
+        }
+
+        public void upsertIfConditionElseInsert(ITupleReference tuple, ITupleAcceptor acceptor)
+                throws HyracksDataException, TreeIndexException {
+            ctx.setOperation(IndexOperation.UPSERT);
+            ctx.acceptor = acceptor;
             btree.upsert(tuple, ctx);
         }
-        
+
         @Override
-		public ITreeIndexCursor createSearchCursor() {
-			IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
-	        return new BTreeRangeSearchCursor(leafFrame, false);
-		}
-        
+        public ITreeIndexCursor createSearchCursor() {
+            IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
+            return new BTreeRangeSearchCursor(leafFrame, false);
+        }
+
         @Override
         public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
                 TreeIndexException {
-            ctx.reset(IndexOp.SEARCH);
+            ctx.setOperation(IndexOperation.SEARCH);
             btree.search((ITreeIndexCursor) cursor, searchPred, ctx);
         }
 
         @Override
-		public ITreeIndexCursor createDiskOrderScanCursor() {
-			IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
-	        return new TreeDiskOrderScanCursor(leafFrame);
-		}
-        
+        public ITreeIndexCursor createDiskOrderScanCursor() {
+            IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
+            return new TreeIndexDiskOrderScanCursor(leafFrame);
+        }
+
         @Override
         public void diskOrderScan(ITreeIndexCursor cursor) throws HyracksDataException {
-            ctx.reset(IndexOp.DISKORDERSCAN);
+            ctx.setOperation(IndexOperation.DISKORDERSCAN);
             btree.diskOrderScan(cursor, ctx);
         }
-		
-		// TODO: Ideally, this method should not exist. But we need it for
-		// the changing the leafFrame and leafFrameFactory of the op context for
-		// the LSM-BTree to work correctly.
-		public BTreeOpContext getOpContext() {
-			return ctx;
-		}
+
+        // TODO: Ideally, this method should not exist. But we need it for
+        // the changing the leafFrame and leafFrameFactory of the op context for
+        // the LSM-BTree to work correctly.
+        public BTreeOpContext getOpContext() {
+            return ctx;
+        }
+
+        public ITreeIndexCursor createCountingSearchCursor() {
+            IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
+            return new BTreeCountingSearchCursor(leafFrame, false);
+        }
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+            throws TreeIndexException {
+        try {
+            return new BTreeBulkLoader(fillFactor, verifyInput);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    public class BTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
+        protected final ISplitKey splitKey;
+        protected final boolean verifyInput;
+
+        public BTreeBulkLoader(float fillFactor, boolean verifyInput) throws TreeIndexException, HyracksDataException {
+            super(fillFactor);
+            this.verifyInput = verifyInput;
+            splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
+            splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
+            try {
+                NodeFrontier leafFrontier = nodeFrontiers.get(0);
+
+                int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
+                int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+
+                // try to free space by compression
+                if (spaceUsed + spaceNeeded > leafMaxBytes) {
+                    leafFrame.compress();
+                    spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+                }
+
+                if (spaceUsed + spaceNeeded > leafMaxBytes) {
+                    leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+                    if (verifyInput) {
+                        verifyInputTuple(tuple, leafFrontier.lastTuple);
+                    }
+                    int splitKeySize = tupleWriter.bytesRequired(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount());
+                    splitKey.initData(splitKeySize);
+                    tupleWriter.writeTupleFields(leafFrontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey
+                            .getBuffer().array(), 0);
+                    splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer(), 0);
+                    splitKey.setLeftPage(leafFrontier.pageId);
+                    leafFrontier.pageId = freePageManager.getFreePage(metaFrame);
+
+                    ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
+                    leafFrontier.page.releaseWriteLatch();
+                    bufferCache.unpin(leafFrontier.page);
+
+                    splitKey.setRightPage(leafFrontier.pageId);
+                    propagateBulk(1);
+
+                    leafFrontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId),
+                            true);
+                    leafFrontier.page.acquireWriteLatch();
+                    leafFrame.setPage(leafFrontier.page);
+                    leafFrame.initBuffer((byte) 0);
+                } else {
+                    if (verifyInput && leafFrame.getTupleCount() > 0) {
+                        leafFrontier.lastTuple.resetByTupleIndex(leafFrame, leafFrame.getTupleCount() - 1);
+                        verifyInputTuple(tuple, leafFrontier.lastTuple);
+                    }
+                }
+
+                leafFrame.setPage(leafFrontier.page);
+                ((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
+            } catch (IndexException e) {
+                handleException();
+                throw e;
+            } catch (HyracksDataException e) {
+                handleException();
+                throw e;
+            } catch (RuntimeException e) {
+                handleException();
+                throw e;
+            }
+        }
+
+        protected void verifyInputTuple(ITupleReference tuple, ITupleReference prevTuple) throws IndexException,
+                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.");
+            }
+        }
+
+        protected void propagateBulk(int level) throws HyracksDataException {
+            if (splitKey.getBuffer() == null)
+                return;
+
+            if (level >= nodeFrontiers.size())
+                addLevel();
+
+            NodeFrontier frontier = nodeFrontiers.get(level);
+            interiorFrame.setPage(frontier.page);
+
+            ITupleReference tuple = splitKey.getTuple();
+            int spaceNeeded = tupleWriter.bytesRequired(tuple, 0, cmp.getKeyFieldCount()) + slotSize + 4;
+            int spaceUsed = interiorFrame.getBuffer().capacity() - interiorFrame.getTotalFreeSpace();
+            if (spaceUsed + spaceNeeded > interiorMaxBytes) {
+
+                ISplitKey copyKey = splitKey.duplicate(leafFrame.getTupleWriter().createTupleReference());
+                tuple = copyKey.getTuple();
+
+                frontier.lastTuple.resetByTupleIndex(interiorFrame, interiorFrame.getTupleCount() - 1);
+                int splitKeySize = tupleWriter.bytesRequired(frontier.lastTuple, 0, cmp.getKeyFieldCount());
+                splitKey.initData(splitKeySize);
+                tupleWriter.writeTupleFields(frontier.lastTuple, 0, cmp.getKeyFieldCount(), splitKey.getBuffer()
+                        .array(), 0);
+                splitKey.getTuple().resetByTupleOffset(splitKey.getBuffer(), 0);
+                splitKey.setLeftPage(frontier.pageId);
+
+                ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
+
+                frontier.page.releaseWriteLatch();
+                bufferCache.unpin(frontier.page);
+                frontier.pageId = freePageManager.getFreePage(metaFrame);
+
+                splitKey.setRightPage(frontier.pageId);
+                propagateBulk(level + 1);
+
+                frontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, frontier.pageId), true);
+                frontier.page.acquireWriteLatch();
+                interiorFrame.setPage(frontier.page);
+                interiorFrame.initBuffer((byte) level);
+            }
+            ((IBTreeInteriorFrame) interiorFrame).insertSorted(tuple);
+        }
+
+    }
+
+    @SuppressWarnings("rawtypes")
+    public static String printLeafFrameTuples(IBTreeLeafFrame leafFrame, ISerializerDeserializer[] fieldSerdes)
+            throws HyracksDataException {
+        StringBuilder strBuilder = new StringBuilder();
+        ITreeIndexTupleReference tuple = leafFrame.createTupleReference();
+        for (int i = 0; i < leafFrame.getTupleCount(); i++) {
+            tuple.resetByTupleIndex(leafFrame, i);
+            String tupleString = TupleUtils.printTuple(tuple, fieldSerdes);
+            strBuilder.append(tupleString + " | ");
+        }
+        // Print right link.
+        int rightPageId = leafFrame.getNextLeaf();
+        strBuilder.append("(" + rightPageId + ")");
+        return strBuilder.toString();
+    }
+
+    @SuppressWarnings("rawtypes")
+    public static String printInteriorFrameTuples(IBTreeInteriorFrame interiorFrame,
+            ISerializerDeserializer[] fieldSerdes) throws HyracksDataException {
+        StringBuilder strBuilder = new StringBuilder();
+        ITreeIndexTupleReference tuple = interiorFrame.createTupleReference();
+        for (int i = 0; i < interiorFrame.getTupleCount(); i++) {
+            tuple.resetByTupleIndex(interiorFrame, i);
+            // Print child pointer.
+            int numFields = tuple.getFieldCount();
+            int childPageId = IntegerSerializerDeserializer.getInt(tuple.getFieldData(numFields - 1),
+                    tuple.getFieldStart(numFields - 1) + tuple.getFieldLength(numFields - 1));
+            strBuilder.append("(" + childPageId + ") ");
+            String tupleString = TupleUtils.printTuple(tuple, fieldSerdes);
+            strBuilder.append(tupleString + " | ");
+        }
+        // Print rightmost pointer.
+        int rightMostChildPageId = interiorFrame.getRightmostChildPageId();
+        strBuilder.append("(" + rightMostChildPageId + ")");
+        return strBuilder.toString();
     }
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
new file mode 100644
index 0000000..0ed1dbe
--- /dev/null
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
@@ -0,0 +1,249 @@
+/*
+ * Copyright 2009-2012 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.btree.impls;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.FindTupleMode;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+
+public class BTreeCountingSearchCursor implements ITreeIndexCursor {
+
+    private int fileId = -1;
+    private ICachedPage page = null;
+    private IBufferCache bufferCache = null;
+
+    private int tupleIndex = 0;
+    private int stopTupleIndex;
+    private int count = -1;
+
+    private FindTupleMode lowKeyFtm;
+    private FindTupleMode highKeyFtm;
+
+    private FindTupleNoExactMatchPolicy lowKeyFtp;
+    private FindTupleNoExactMatchPolicy highKeyFtp;
+
+    private final IBTreeLeafFrame frame;
+    private final ITreeIndexTupleReference frameTuple;
+    private final boolean exclusiveLatchNodes;
+
+    private RangePredicate pred;
+    private MultiComparator lowKeyCmp;
+    private MultiComparator highKeyCmp;
+    private ITupleReference lowKey;
+    private ITupleReference highKey;
+
+    // For storing the count.
+    private byte[] countBuf = new byte[4];
+    private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1);
+    private ArrayTupleReference countTuple = new ArrayTupleReference();    
+    
+    public BTreeCountingSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
+        this.frame = frame;
+        this.frameTuple = frame.createTupleReference();
+        this.exclusiveLatchNodes = exclusiveLatchNodes;
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        // in case open is called multiple times without closing
+        if (page != null) {
+            if (exclusiveLatchNodes) {
+                page.releaseWriteLatch();
+            } else {
+                page.releaseReadLatch();
+            }
+            bufferCache.unpin(page);
+        }
+
+        page = ((BTreeCursorInitialState) initialState).getPage();
+        frame.setPage(page);
+
+        pred = (RangePredicate) searchPred;
+        lowKeyCmp = pred.getLowKeyComparator();
+        highKeyCmp = pred.getHighKeyComparator();
+
+        lowKey = pred.getLowKey();
+        highKey = pred.getHighKey();
+
+        // init
+        lowKeyFtm = FindTupleMode.EXCLUSIVE;
+        if (pred.lowKeyInclusive) {
+            lowKeyFtp = FindTupleNoExactMatchPolicy.LOWER_KEY;
+        } else {
+            lowKeyFtp = FindTupleNoExactMatchPolicy.HIGHER_KEY;
+        }
+
+        highKeyFtm = FindTupleMode.EXCLUSIVE;
+        if (pred.highKeyInclusive) {
+            highKeyFtp = FindTupleNoExactMatchPolicy.HIGHER_KEY;
+        } else {
+            highKeyFtp = FindTupleNoExactMatchPolicy.LOWER_KEY;
+        }
+
+        tupleIndex = getLowKeyIndex();
+        stopTupleIndex = getHighKeyIndex();        
+    }
+
+    private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException {
+        do {
+            ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
+            if (exclusiveLatchNodes) {
+                nextLeaf.acquireWriteLatch();
+                page.releaseWriteLatch();
+            } else {
+                nextLeaf.acquireReadLatch();
+                page.releaseReadLatch();
+            }
+            bufferCache.unpin(page);
+            page = nextLeaf;
+            frame.setPage(page);
+            nextLeafPage = frame.getNextLeaf();
+        } while (frame.getTupleCount() == 0 && nextLeafPage > 0);
+    }
+
+    private int getLowKeyIndex() throws HyracksDataException {
+        if (lowKey == null) {
+            return 0;
+        }
+        int index = frame.findTupleIndex(lowKey, frameTuple, lowKeyCmp, lowKeyFtm, lowKeyFtp);
+        if (pred.lowKeyInclusive) {
+            index++;
+        } else {
+            if (index < 0) {
+                index = frame.getTupleCount();
+            }
+        }
+        return index;
+    }
+
+    private int getHighKeyIndex() throws HyracksDataException {
+        if (highKey == null) {
+            return frame.getTupleCount() - 1;
+        }
+        int index = frame.findTupleIndex(highKey, frameTuple, highKeyCmp, highKeyFtm, highKeyFtp);
+        if (pred.highKeyInclusive) {
+            if (index < 0) {
+                index = frame.getTupleCount() - 1;
+            } else {
+                index--;
+            }
+        }
+        return index;
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException {
+        // get the count for the current page
+        // follow the sibling pointer until last page
+        // if no more tuples on a page, then done
+
+        if (count < 0) {
+            count = 0;
+
+            while (stopTupleIndex >= 0 || frame.getTupleCount() == 0) {
+                count += (stopTupleIndex - tupleIndex + 1);
+
+                int nextLeafPage = frame.getNextLeaf();
+                if (nextLeafPage >= 0) {
+                    fetchNextLeafPage(nextLeafPage);
+                } else {
+                    // No more pages. Done counting!
+                    break;
+                }
+
+                tupleIndex = 0;
+                stopTupleIndex = getHighKeyIndex();
+            }
+
+            return true;
+        }
+
+        return false;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        // Do nothing. Count is performed just once!
+        IntegerSerializerDeserializer.putInt(count, countBuf, 0);
+        tupleBuilder.addField(countBuf, 0, 4);
+        countTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (page != null) {
+            if (exclusiveLatchNodes) {
+                page.releaseWriteLatch();
+            } else {
+                page.releaseReadLatch();
+            }
+            bufferCache.unpin(page);
+        }
+        tupleBuilder.reset();
+        tupleIndex = 0;
+        page = null;
+        pred = null;
+        count = -1;
+    }
+
+    @Override
+    public void reset() {
+        try {
+            close();
+        } catch (Exception e) {
+            e.printStackTrace();
+        }        
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return countTuple;
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        return page;
+    }
+
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        this.bufferCache = bufferCache;
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        this.fileId = fileId;
+    }
+
+    @Override
+    public boolean exclusiveLatchNodes() {
+        return exclusiveLatchNodes;
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCursorInitialState.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCursorInitialState.java
index 855f9e6..9d7b612 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCursorInitialState.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCursorInitialState.java
@@ -1,14 +1,28 @@
 package edu.uci.ics.hyracks.storage.am.btree.impls;
 
 import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 
 public class BTreeCursorInitialState implements ICursorInitialState {
 
+    // This is only used by the LSM-RTree
+    private int pageId;
     private ICachedPage page;
+    private ISearchOperationCallback searchCallback;
+    private MultiComparator originalKeyCmp;
+    private final IIndexAccessor accessor;
 
-    public BTreeCursorInitialState(ICachedPage page) {
+    public BTreeCursorInitialState(ICachedPage page, ISearchOperationCallback searchCallback, IIndexAccessor accessor) {
         this.page = page;
+        this.searchCallback = searchCallback;
+        this.accessor = accessor;
+    }
+    
+    public IIndexAccessor getAccessor() {
+        return accessor;
     }
 
     public ICachedPage getPage() {
@@ -18,4 +32,32 @@
     public void setPage(ICachedPage page) {
         this.page = page;
     }
+
+    public int getPageId() {
+        return pageId;
+    }
+
+    public void setPageId(int pageId) {
+        this.pageId = pageId;
+    }
+
+    @Override
+    public ISearchOperationCallback getSearchOperationCallback() {
+        return searchCallback;
+    }
+
+    @Override
+    public void setSearchOperationCallback(ISearchOperationCallback searchCallback) {
+        this.searchCallback = searchCallback;
+    }
+
+    @Override
+    public MultiComparator getOriginalKeyComparator() {
+        return originalKeyCmp;
+    }
+
+    @Override
+    public void setOriginialKeyComparator(MultiComparator originalCmp) {
+        this.originalKeyCmp = originalCmp;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
index 23a14eb..6991850 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
@@ -15,52 +15,94 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.impls;
 
+import java.util.ArrayDeque;
+import java.util.Deque;
+
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.btree.api.ITupleAcceptor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IntArrayList;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.LongArrayList;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
-public class BTreeOpContext implements IIndexOpContext {
+public class BTreeOpContext implements IIndexOperationContext {
     private final int INIT_ARRAYLIST_SIZE = 6;
+
+    public IIndexAccessor accessor;
     public MultiComparator cmp;
     public ITreeIndexFrameFactory leafFrameFactory;
     public ITreeIndexFrameFactory interiorFrameFactory;
     public IBTreeLeafFrame leafFrame;
     public IBTreeInteriorFrame interiorFrame;
     public ITreeIndexMetaDataFrame metaFrame;
-    public IndexOp op;
+    public IndexOperation op;
     public ITreeIndexCursor cursor;
     public BTreeCursorInitialState cursorInitialState;
     public RangePredicate pred;
-    public BTreeSplitKey splitKey;    
+    public BTreeSplitKey splitKey;
     public LongArrayList pageLsns;
     public IntArrayList smPages;
     public IntArrayList freePages;
     public int opRestarts = 0;
     public boolean exceptionHandled;
-    
-    public BTreeOpContext(ITreeIndexFrameFactory leafFrameFactory, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexMetaDataFrame metaFrame, IBinaryComparatorFactory[] cmpFactories) {
-        this.cmp = MultiComparator.create(cmpFactories);
+    public IModificationOperationCallback modificationCallback;
+    public ISearchOperationCallback searchCallback;
+    public ITupleAcceptor acceptor;
+    public int smoCount;
+
+    // Debug
+    public final Deque<PageValidationInfo> validationInfos;
+    public final ITreeIndexTupleReference interiorFrameTuple;
+    public final ITreeIndexTupleReference leafFrameTuple;
+
+    public BTreeOpContext(IIndexAccessor accessor, ITreeIndexFrameFactory leafFrameFactory,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexMetaDataFrame metaFrame,
+            IBinaryComparatorFactory[] cmpFactories, IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        this.accessor = accessor;
+
+        if (cmpFactories[0] != null) {
+            //            this.cmp = MultiComparator.createIgnoreFieldLength(cmpFactories);
+            this.cmp = MultiComparator.create(cmpFactories);
+        } else {
+            this.cmp = null;
+        }
+
         this.leafFrameFactory = leafFrameFactory;
         this.leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
-        if (leafFrame != null) {
+        if (leafFrame != null && this.cmp != null) {
             leafFrame.setMultiComparator(cmp);
         }
         this.interiorFrameFactory = interiorFrameFactory;
         this.interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
-        if (interiorFrame != null) {
+        if (interiorFrame != null && this.cmp != null) {
             interiorFrame.setMultiComparator(cmp);
         }
         this.metaFrame = metaFrame;
         this.pageLsns = new LongArrayList(INIT_ARRAYLIST_SIZE, INIT_ARRAYLIST_SIZE);
+        this.smoCount = 0;
+        this.modificationCallback = modificationCallback;
+        this.searchCallback = searchCallback;
+
+        // Debug
+        this.validationInfos = new ArrayDeque<PageValidationInfo>(INIT_ARRAYLIST_SIZE);
+        this.interiorFrameTuple = interiorFrame.createTupleReference();
+        this.leafFrameTuple = leafFrame.createTupleReference();
     }
 
     public void reset() {
@@ -71,14 +113,15 @@
         if (smPages != null)
             smPages.clear();
         opRestarts = 0;
+        smoCount = 0;
         exceptionHandled = false;
     }
 
     @Override
-    public void reset(IndexOp newOp) {
-        if (newOp == IndexOp.SEARCH || newOp == IndexOp.DISKORDERSCAN) {
+    public void setOperation(IndexOperation newOp) {
+        if (newOp == IndexOperation.SEARCH || newOp == IndexOperation.DISKORDERSCAN) {
             if (cursorInitialState == null) {
-                cursorInitialState = new BTreeCursorInitialState(null);
+                cursorInitialState = new BTreeCursorInitialState(null, searchCallback, accessor);
             }
         } else {
             // Insert, delete, update or upsert operation.
@@ -96,6 +139,7 @@
             }
         }
         op = newOp;
+        smoCount = 0;
         exceptionHandled = false;
     }
 
@@ -106,4 +150,77 @@
     public IBTreeInteriorFrame createInteriorFrame() {
         return (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
     }
+
+    public PageValidationInfo createPageValidationInfo(PageValidationInfo parent) throws HyracksDataException {
+        return new PageValidationInfo(parent);
+    }
+
+    public class PageValidationInfo {
+        public final int numKeyFields;
+
+        public final ArrayTupleBuilder lowRangeBuilder;
+        public final ArrayTupleBuilder highRangeBuilder;
+        public final ArrayTupleReference lowRangeTuple;
+        public final ArrayTupleReference highRangeTuple;
+
+        public boolean isLowRangeNull;
+        public boolean isHighRangeNull;
+
+        public PageValidationInfo() {
+            this.numKeyFields = cmp.getKeyFieldCount();
+            this.lowRangeBuilder = new ArrayTupleBuilder(numKeyFields);
+            this.highRangeBuilder = new ArrayTupleBuilder(numKeyFields);
+            this.lowRangeTuple = new ArrayTupleReference();
+            this.highRangeTuple = new ArrayTupleReference();
+            this.isLowRangeNull = true;
+            this.isHighRangeNull = true;
+        }
+
+        public PageValidationInfo(PageValidationInfo copy) throws HyracksDataException {
+            this();
+            if (copy != null) {
+                propagateLowRangeKey(copy);
+                propagateHighRangeKey(copy);
+            }
+        }
+
+        public void propagateLowRangeKey(PageValidationInfo toPropagate) throws HyracksDataException {
+            isLowRangeNull = toPropagate.isLowRangeNull;
+            if (!isLowRangeNull) {
+                adjustRangeKey(lowRangeBuilder, lowRangeTuple, toPropagate.lowRangeTuple);
+            }
+        }
+
+        public void propagateHighRangeKey(PageValidationInfo toPropagate) throws HyracksDataException {
+            isHighRangeNull = toPropagate.isHighRangeNull;
+            if (!isHighRangeNull) {
+                adjustRangeKey(highRangeBuilder, highRangeTuple, toPropagate.highRangeTuple);
+            }
+        }
+
+        public void adjustLowRangeKey(ITupleReference newLowRangeKey) throws HyracksDataException {
+            isLowRangeNull = newLowRangeKey == null ? true : false;
+            if (!isLowRangeNull) {
+                adjustRangeKey(lowRangeBuilder, lowRangeTuple, newLowRangeKey);
+            }
+        }
+
+        public void adjustHighRangeKey(ITupleReference newHighRangeKey) throws HyracksDataException {
+            isHighRangeNull = newHighRangeKey == null ? true : false;
+            if (!isHighRangeNull) {
+                adjustRangeKey(highRangeBuilder, highRangeTuple, newHighRangeKey);
+            }
+        }
+
+        private void adjustRangeKey(ArrayTupleBuilder builder, ArrayTupleReference tuple, ITupleReference newRangeKey)
+                throws HyracksDataException {
+            TupleUtils.copyTuple(builder, newRangeKey, numKeyFields);
+            tuple.reset(builder.getFieldEndOffsets(), builder.getByteArray());
+        }
+    }
+
+    @Override
+    public IndexOperation getOperation() {
+        return op;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index 8bf4db3..607e00a 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -16,12 +16,18 @@
 package edu.uci.ics.hyracks.storage.am.btree.impls;
 
 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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.FindTupleMode;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
@@ -31,34 +37,43 @@
 
 public class BTreeRangeSearchCursor implements ITreeIndexCursor {
 
-    private int fileId = -1;
-    private ICachedPage page = null;
-    private IBufferCache bufferCache = null;
-
-    private int tupleIndex = 0;
-    private int stopTupleIndex;
-    private int tupleIndexInc = 0;
-
-    private FindTupleMode lowKeyFtm;
-    private FindTupleMode highKeyFtm;
-
-    private FindTupleNoExactMatchPolicy lowKeyFtp;
-    private FindTupleNoExactMatchPolicy highKeyFtp;
-
     private final IBTreeLeafFrame frame;
     private final ITreeIndexTupleReference frameTuple;
     private final boolean exclusiveLatchNodes;
 
+    private IBufferCache bufferCache = null;
+    private int fileId = -1;
+
+    private ICachedPage page = null;
+    private int pageId = -1; // This is used by the LSMRTree flush operation
+
+    private int tupleIndex = 0;
+    private int stopTupleIndex;
+
+    private final RangePredicate reusablePredicate;
+    private final ArrayTupleReference reconciliationTuple;
+    private IIndexAccessor accessor;
+    private ISearchOperationCallback searchCb;
+    private MultiComparator originalKeyCmp;
+    private ArrayTupleBuilder tupleBuilder;
+
+    private FindTupleMode lowKeyFtm;
+    private FindTupleMode highKeyFtm;
+    private FindTupleNoExactMatchPolicy lowKeyFtp;
+    private FindTupleNoExactMatchPolicy highKeyFtp;
+
     private RangePredicate pred;
     private MultiComparator lowKeyCmp;
     private MultiComparator highKeyCmp;
-    private ITupleReference lowKey;
+    protected ITupleReference lowKey;
     private ITupleReference highKey;
 
     public BTreeRangeSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
         this.frame = frame;
         this.frameTuple = frame.createTupleReference();
         this.exclusiveLatchNodes = exclusiveLatchNodes;
+        this.reusablePredicate = new RangePredicate();
+        this.reconciliationTuple = new ArrayTupleReference();
     }
 
     @Override
@@ -71,6 +86,7 @@
             }
             bufferCache.unpin(page);
         }
+
         tupleIndex = 0;
         page = null;
         pred = null;
@@ -85,6 +101,14 @@
         return page;
     }
 
+    public int getTupleOffset() {
+        return frame.getTupleOffset(tupleIndex - 1);
+    }
+
+    public int getPageId() {
+        return pageId;
+    }
+
     private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException {
         do {
             ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
@@ -96,16 +120,19 @@
                 page.releaseReadLatch();
             }
             bufferCache.unpin(page);
+
             page = nextLeaf;
             frame.setPage(page);
+            pageId = nextLeafPage;
             nextLeafPage = frame.getNextLeaf();
         } while (frame.getTupleCount() == 0 && nextLeafPage > 0);
     }
 
     @Override
     public boolean hasNext() throws HyracksDataException {
+        int nextLeafPage;
         if (tupleIndex >= frame.getTupleCount()) {
-            int nextLeafPage = frame.getNextLeaf();
+            nextLeafPage = frame.getNextLeaf();
             if (nextLeafPage >= 0) {
                 fetchNextLeafPage(nextLeafPage);
                 tupleIndex = 0;
@@ -118,23 +145,67 @@
             }
         }
 
-        frameTuple.resetByTupleIndex(frame, tupleIndex);
-        if (highKey == null || tupleIndex <= stopTupleIndex) {
-            return true;
-        } else {
+        if (tupleIndex > stopTupleIndex) {
             return false;
         }
+
+        frameTuple.resetByTupleIndex(frame, tupleIndex);
+        while (true) {
+            if (searchCb.proceed(frameTuple)) {
+                return true;
+            } else {
+                // copy the tuple before we unlatch/unpin
+                if (tupleBuilder == null) {
+                    tupleBuilder = new ArrayTupleBuilder(originalKeyCmp.getKeyFieldCount());
+                }
+                TupleUtils.copyTuple(tupleBuilder, frameTuple, originalKeyCmp.getKeyFieldCount());
+                reconciliationTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+
+                // unlatch/unpin
+                if (exclusiveLatchNodes) {
+                    page.releaseWriteLatch();
+                } else {
+                    page.releaseReadLatch();
+                }
+                bufferCache.unpin(page);
+                page = null;
+
+                // reconcile
+                searchCb.reconcile(reconciliationTuple);
+
+                // retraverse the index looking for the reconciled key
+                reusablePredicate.setLowKey(reconciliationTuple, true);
+                try {
+                    accessor.search(this, reusablePredicate);
+                } catch (IndexException e) {
+                    throw new HyracksDataException(e);
+                }
+
+                if (stopTupleIndex < 0 || tupleIndex > stopTupleIndex) {
+                    return false;
+                }
+
+                // see if we found the tuple we reconciled on
+                frameTuple.resetByTupleIndex(frame, tupleIndex);
+                if (originalKeyCmp.compare(reconciliationTuple, frameTuple) == 0) {
+                    return true;
+                } else {
+                    searchCb.cancel(reconciliationTuple);
+                }
+            }
+        }
     }
-    
+
     @Override
     public void next() throws HyracksDataException {
-        tupleIndex += tupleIndexInc;
+        tupleIndex++;
     }
 
     private int getLowKeyIndex() throws HyracksDataException {
         if (lowKey == null) {
             return 0;
         }
+
         int index = frame.findTupleIndex(lowKey, frameTuple, lowKeyCmp, lowKeyFtm, lowKeyFtp);
         if (pred.lowKeyInclusive) {
             index++;
@@ -143,6 +214,7 @@
                 index = frame.getTupleCount();
             }
         }
+
         return index;
     }
 
@@ -150,15 +222,16 @@
         if (highKey == null) {
             return frame.getTupleCount() - 1;
         }
+
         int index = frame.findTupleIndex(highKey, frameTuple, highKeyCmp, highKeyFtm, highKeyFtp);
         if (pred.highKeyInclusive) {
             if (index < 0) {
                 index = frame.getTupleCount() - 1;
-            }
-            else {
+            } else {
                 index--;
             }
         }
+
         return index;
     }
 
@@ -173,18 +246,23 @@
             }
             bufferCache.unpin(page);
         }
-
-        page = ((BTreeCursorInitialState) initialState).getPage();
+        accessor = ((BTreeCursorInitialState) initialState).getAccessor();
+        searchCb = initialState.getSearchOperationCallback();
+        originalKeyCmp = initialState.getOriginalKeyComparator();
+        pageId = ((BTreeCursorInitialState) initialState).getPageId();
+        page = initialState.getPage();
         frame.setPage(page);
 
         pred = (RangePredicate) searchPred;
         lowKeyCmp = pred.getLowKeyComparator();
         highKeyCmp = pred.getHighKeyComparator();
-
         lowKey = pred.getLowKey();
         highKey = pred.getHighKey();
 
-        // init
+        reusablePredicate.setLowKeyComparator(originalKeyCmp);
+        reusablePredicate.setHighKeyComparator(pred.getHighKeyComparator());
+        reusablePredicate.setHighKey(pred.getHighKey(), pred.isHighKeyInclusive());
+
         lowKeyFtm = FindTupleMode.EXCLUSIVE;
         if (pred.lowKeyInclusive) {
             lowKeyFtp = FindTupleNoExactMatchPolicy.LOWER_KEY;
@@ -198,19 +276,14 @@
         } else {
             highKeyFtp = FindTupleNoExactMatchPolicy.LOWER_KEY;
         }
-        
+
         tupleIndex = getLowKeyIndex();
         stopTupleIndex = getHighKeyIndex();
-        tupleIndexInc = 1;
     }
 
     @Override
-    public void reset() {
-        try {
-            close();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
+    public void reset() throws HyracksDataException {
+        close();
     }
 
     @Override
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeSplitKey.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeSplitKey.java
index e664e5b..2606c08 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeSplitKey.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeSplitKey.java
@@ -21,15 +21,16 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 
 public class BTreeSplitKey implements ISplitKey {
+    public final ITreeIndexTupleReference tuple;
+
     public byte[] data = null;
     public ByteBuffer buf = null;
-    public ITreeIndexTupleReference tuple;
     public int keySize = 0;
 
     public BTreeSplitKey(ITreeIndexTupleReference tuple) {
         this.tuple = tuple;
     }
-    
+
     public void initData(int keySize) {
         // try to reuse existing memory from a lower-level split if possible
         this.keySize = keySize;
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
index 80ac173..44fcdef 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixPrefixTupleReference.java
@@ -16,6 +16,7 @@
 package edu.uci.ics.hyracks.storage.am.btree.impls;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.btree.api.IPrefixSlotManager;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleReference;
@@ -30,10 +31,11 @@
     @Override
     public void resetByTupleIndex(ITreeIndexFrame frame, int tupleIndex) {
         BTreeFieldPrefixNSMLeafFrame concreteFrame = (BTreeFieldPrefixNSMLeafFrame) frame;
-        int prefixSlotOff = concreteFrame.slotManager.getPrefixSlotOff(tupleIndex);
+        IPrefixSlotManager slotManager = concreteFrame.getSlotManager();
+        int prefixSlotOff = slotManager.getPrefixSlotOff(tupleIndex);
         int prefixSlot = concreteFrame.getBuffer().getInt(prefixSlotOff);
-        setFieldCount(concreteFrame.slotManager.decodeFirstSlotField(prefixSlot));
-        tupleStartOff = concreteFrame.slotManager.decodeSecondSlotField(prefixSlot);
+        setFieldCount(slotManager.decodeFirstSlotField(prefixSlot));
+        tupleStartOff = slotManager.decodeSecondSlotField(prefixSlot);
         buf = concreteFrame.getBuffer();
         resetByTupleOffset(buf, tupleStartOff);
     }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixSlotManager.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixSlotManager.java
index 6525bb9..4c66fbb 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixSlotManager.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixSlotManager.java
@@ -28,16 +28,17 @@
 
 public class FieldPrefixSlotManager implements IPrefixSlotManager {
 
-    private static final int slotSize = 4;
     public static final int TUPLE_UNCOMPRESSED = 0xFF;
     public static final int MAX_PREFIX_SLOTS = 0xFE;
     public static final int GREATEST_KEY_INDICATOR = 0x00FFFFFF;
     public static final int ERROR_INDICATOR = 0x00FFFFFE;
 
+    private static final int slotSize = 4;
+
     private ByteBuffer buf;
     private BTreeFieldPrefixNSMLeafFrame frame;
     private MultiComparator cmp;
-    
+
     public int decodeFirstSlotField(int slot) {
         return (slot & 0xFF000000) >>> 24;
     }
@@ -117,23 +118,17 @@
             }
         }
 
-        // System.out.println("SLOTLBOUND: " + tuplePrefixSlotNumLbound);
-        // System.out.println("SLOTUBOUND: " + tuplePrefixSlotNumUbound);
-
         int tupleMid = -1;
         int tupleBegin = 0;
         int tupleEnd = frame.getTupleCount() - 1;
 
-        // binary search on tuples, guided by the lower and upper bounds on
-        // prefixSlotNum
+        // binary search on tuples, guided by the lower and upper bounds on prefixSlotNum
         while (tupleBegin <= tupleEnd) {
             tupleMid = (tupleBegin + tupleEnd) / 2;
             int tupleSlotOff = getTupleSlotOff(tupleMid);
             int tupleSlot = buf.getInt(tupleSlotOff);
             int prefixSlotNum = decodeFirstSlotField(tupleSlot);
 
-            // System.out.println("RECS: " + recBegin + " " + recMid + " " +
-            // recEnd);
             int cmp = 0;
             if (prefixSlotNum == TUPLE_UNCOMPRESSED) {
                 frameTuple.resetByTupleIndex(frame, tupleMid);
@@ -160,18 +155,15 @@
                     else
                         tupleEnd = tupleMid - 1;
                 } else {
-                	if (mode == FindTupleMode.EXCLUSIVE_ERROR_IF_EXISTS) {
-                		return encodeSlotFields(prefixMatch, ERROR_INDICATOR);
-                	} else {
-                		return encodeSlotFields(prefixMatch, tupleMid);
-                	}
+                    if (mode == FindTupleMode.EXCLUSIVE_ERROR_IF_EXISTS) {
+                        return encodeSlotFields(prefixMatch, ERROR_INDICATOR);
+                    } else {
+                        return encodeSlotFields(prefixMatch, tupleMid);
+                    }
                 }
             }
         }
 
-        // System.out.println("RECS: " + recBegin + " " + recMid + " " +
-        // recEnd);
-
         if (mode == FindTupleMode.EXACT)
             return encodeSlotFields(prefixMatch, ERROR_INDICATOR);
 
@@ -223,7 +215,7 @@
     public int insertSlot(int slot, int tupleOff) {
         int slotNum = decodeSecondSlotField(slot);
         if (slotNum == ERROR_INDICATOR) {
-        	System.out.println("WOW BIG PROBLEM!");
+            System.out.println("WOW BIG PROBLEM!");
         }
         if (slotNum == GREATEST_KEY_INDICATOR) {
             int slotOff = getTupleSlotEndOff() - slotSize;
@@ -255,50 +247,49 @@
         buf.putInt(getPrefixSlotOff(tupleIndex), slot);
     }
 
-	@Override
-	public int getGreatestKeyIndicator() {
-		return GREATEST_KEY_INDICATOR;
-	}
+    @Override
+    public int getGreatestKeyIndicator() {
+        return GREATEST_KEY_INDICATOR;
+    }
 
-	@Override
-	public int getErrorIndicator() {
-		return ERROR_INDICATOR;
-	}
+    @Override
+    public int getErrorIndicator() {
+        return ERROR_INDICATOR;
+    }
 
-	@Override
-	public void setFrame(ITreeIndexFrame frame) {
-		this.frame = (BTreeFieldPrefixNSMLeafFrame)frame;
+    @Override
+    public void setFrame(ITreeIndexFrame frame) {
+        this.frame = (BTreeFieldPrefixNSMLeafFrame) frame;
         this.buf = frame.getBuffer();
-	}
+    }
 
-	@Override
-	public int findTupleIndex(ITupleReference searchKey,
-			ITreeIndexTupleReference frameTuple, MultiComparator multiCmp,
-			FindTupleMode mode, FindTupleNoExactMatchPolicy matchPolicy) {
-		throw new UnsupportedOperationException("Not implemented.");
-	}
-	
-	@Override
-	public int getSlotStartOff() {
-		throw new UnsupportedOperationException("Not implemented.");
-	}
+    @Override
+    public int findTupleIndex(ITupleReference searchKey, ITreeIndexTupleReference frameTuple, MultiComparator multiCmp,
+            FindTupleMode mode, FindTupleNoExactMatchPolicy matchPolicy) {
+        throw new UnsupportedOperationException("Not implemented.");
+    }
 
-	@Override
-	public int getSlotEndOff() {
-		throw new UnsupportedOperationException("Not implemented.");
-	}
+    @Override
+    public int getSlotStartOff() {
+        throw new UnsupportedOperationException("Not implemented.");
+    }
 
-	@Override
-	public int getTupleOff(int slotOff) {
-		throw new UnsupportedOperationException("Not implemented.");
-	}
+    @Override
+    public int getSlotEndOff() {
+        throw new UnsupportedOperationException("Not implemented.");
+    }
 
-	@Override
-	public int getSlotOff(int tupleIndex) {
-		throw new UnsupportedOperationException("Not implemented.");
-	}
-	
-	public void setMultiComparator(MultiComparator cmp) {
-		this.cmp = cmp;
-	}
+    @Override
+    public int getTupleOff(int slotOff) {
+        throw new UnsupportedOperationException("Not implemented.");
+    }
+
+    @Override
+    public int getSlotOff(int tupleIndex) {
+        throw new UnsupportedOperationException("Not implemented.");
+    }
+
+    public void setMultiComparator(MultiComparator cmp) {
+        this.cmp = cmp;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixTupleReference.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixTupleReference.java
index 9644a3e..b7174d4 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixTupleReference.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/FieldPrefixTupleReference.java
@@ -2,18 +2,20 @@
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.storage.am.btree.api.IPrefixSlotManager;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 
 public class FieldPrefixTupleReference implements ITreeIndexTupleReference {
 
+    private final ITreeIndexTupleReference helperTuple;
+
     private BTreeFieldPrefixNSMLeafFrame frame;
     private int prefixTupleStartOff;
     private int suffixTupleStartOff;
     private int numPrefixFields;
     private int fieldCount;
-    private ITreeIndexTupleReference helperTuple;
 
     public FieldPrefixTupleReference(ITreeIndexTupleReference helperTuple) {
         this.helperTuple = helperTuple;
@@ -23,17 +25,17 @@
     @Override
     public void resetByTupleIndex(ITreeIndexFrame frame, int tupleIndex) {
         this.frame = (BTreeFieldPrefixNSMLeafFrame) frame;
-
-        int tupleSlotOff = this.frame.slotManager.getTupleSlotOff(tupleIndex);
+        IPrefixSlotManager slotManager = this.frame.getSlotManager();
+        int tupleSlotOff = slotManager.getTupleSlotOff(tupleIndex);
         int tupleSlot = this.frame.getBuffer().getInt(tupleSlotOff);
-        int prefixSlotNum = this.frame.slotManager.decodeFirstSlotField(tupleSlot);
-        suffixTupleStartOff = this.frame.slotManager.decodeSecondSlotField(tupleSlot);
+        int prefixSlotNum = slotManager.decodeFirstSlotField(tupleSlot);
+        suffixTupleStartOff = slotManager.decodeSecondSlotField(tupleSlot);
 
         if (prefixSlotNum != FieldPrefixSlotManager.TUPLE_UNCOMPRESSED) {
-            int prefixSlotOff = this.frame.slotManager.getPrefixSlotOff(prefixSlotNum);
+            int prefixSlotOff = slotManager.getPrefixSlotOff(prefixSlotNum);
             int prefixSlot = this.frame.getBuffer().getInt(prefixSlotOff);
-            numPrefixFields = this.frame.slotManager.decodeFirstSlotField(prefixSlot);
-            prefixTupleStartOff = this.frame.slotManager.decodeSecondSlotField(prefixSlot);
+            numPrefixFields = slotManager.decodeFirstSlotField(prefixSlot);
+            prefixTupleStartOff = slotManager.decodeSecondSlotField(prefixSlot);
         } else {
             numPrefixFields = 0;
             prefixTupleStartOff = -1;
@@ -96,20 +98,21 @@
     public int getTupleSize() {
         return getSuffixTupleSize() + getPrefixTupleSize();
     }
-    
+
     public int getSuffixTupleSize() {
         helperTuple.setFieldCount(numPrefixFields, fieldCount - numPrefixFields);
         helperTuple.resetByTupleOffset(frame.getBuffer(), suffixTupleStartOff);
         return helperTuple.getTupleSize();
     }
-    
+
     public int getPrefixTupleSize() {
-        if (numPrefixFields == 0) return 0;
+        if (numPrefixFields == 0)
+            return 0;
         helperTuple.setFieldCount(numPrefixFields);
         helperTuple.resetByTupleOffset(frame.getBuffer(), prefixTupleStartOff);
         return helperTuple.getTupleSize();
     }
-    
+
     public int getNumPrefixFields() {
         return numPrefixFields;
     }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/UnconditionalTupleAcceptor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/UnconditionalTupleAcceptor.java
new file mode 100644
index 0000000..9ae2523
--- /dev/null
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/UnconditionalTupleAcceptor.java
@@ -0,0 +1,14 @@
+package edu.uci.ics.hyracks.storage.am.btree.impls;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.api.ITupleAcceptor;
+
+public enum UnconditionalTupleAcceptor implements ITupleAcceptor {
+    INSTANCE;
+
+    @Override
+    public boolean accept(ITupleReference tuple) {
+        return true;
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
index 57e3a96..de73459 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
@@ -3,6 +3,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 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.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeFieldPrefixNSMLeafFrameFactory;
@@ -11,7 +12,6 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
@@ -20,22 +20,38 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class BTreeUtils {
-    public static BTree createBTree(IBufferCache bufferCache, IOperationCallback opCallback, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, BTreeLeafFrameType leafType) throws BTreeException {
+    public static BTree createBTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, BTreeLeafFrameType leafType,
+            FileReference file) throws BTreeException {
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = getLeafFrameFactory(tupleWriterFactory, leafType);
         ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
         ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
-        BTree btree = new BTree(bufferCache, opCallback, typeTraits.length, cmpFactories, freePageManager, interiorFrameFactory, leafFrameFactory);
+        BTree btree = new BTree(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory,
+                cmpFactories, typeTraits.length, file);
         return btree;
     }
     
+    public static BTree createBTree(IBufferCache bufferCache, IFreePageManager freePageManager,
+            IFileMapProvider fileMapProvider, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            BTreeLeafFrameType leafType, FileReference file) throws BTreeException {
+        TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+        ITreeIndexFrameFactory leafFrameFactory = getLeafFrameFactory(tupleWriterFactory, leafType);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+        BTree btree = new BTree(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory,
+                cmpFactories, typeTraits.length, file);
+        return btree;
+    }
+
     // Creates a new MultiComparator by constructing new IBinaryComparators.
-    public static MultiComparator getSearchMultiComparator(IBinaryComparatorFactory[] cmpFactories, ITupleReference searchKey) {
+    public static MultiComparator getSearchMultiComparator(IBinaryComparatorFactory[] cmpFactories,
+            ITupleReference searchKey) {
         if (searchKey == null || cmpFactories.length == searchKey.getFieldCount()) {
-            return MultiComparator.create(cmpFactories);
+            return MultiComparator.createIgnoreFieldLength(cmpFactories);
         }
         IBinaryComparator[] newCmps = new IBinaryComparator[searchKey.getFieldCount()];
         for (int i = 0; i < searchKey.getFieldCount(); i++) {
@@ -43,11 +59,12 @@
         }
         return new MultiComparator(newCmps);
     }
-    
-    public static ITreeIndexFrameFactory getLeafFrameFactory(ITreeIndexTupleWriterFactory tupleWriterFactory, BTreeLeafFrameType leafType) throws BTreeException {
-        switch(leafType) {
+
+    public static ITreeIndexFrameFactory getLeafFrameFactory(ITreeIndexTupleWriterFactory tupleWriterFactory,
+            BTreeLeafFrameType leafType) throws BTreeException {
+        switch (leafType) {
             case REGULAR_NSM: {
-                return new BTreeNSMLeafFrameFactory(tupleWriterFactory);                
+                return new BTreeNSMLeafFrameFactory(tupleWriterFactory);
             }
             case FIELD_PREFIX_COMPRESSED_NSM: {
                 return new BTreeFieldPrefixNSMLeafFrameFactory(tupleWriterFactory);
diff --git a/hyracks/hyracks-storage-am-common/pom.xml b/hyracks/hyracks-storage-am-common/pom.xml
index dbc4f41d..aaa64fb 100644
--- a/hyracks/hyracks-storage-am-common/pom.xml
+++ b/hyracks/hyracks-storage-am-common/pom.xml
@@ -1,8 +1,6 @@
 <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>hyracks-storage-am-common</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-storage-am-common</name>
 
   <parent>
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ICursorInitialState.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ICursorInitialState.java
index 60e8ba9..9fbaac2 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ICursorInitialState.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ICursorInitialState.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2009-2010 by The Regents of the University of California
+ * Copyright 2009-2012 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
@@ -15,9 +15,19 @@
 
 package edu.uci.ics.hyracks.storage.am.common.api;
 
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 
 public interface ICursorInitialState {
-	public ICachedPage getPage();
-	public void setPage(ICachedPage page);
+    public ICachedPage getPage();
+
+    public void setPage(ICachedPage page);
+
+    public ISearchOperationCallback getSearchOperationCallback();
+
+    public void setSearchOperationCallback(ISearchOperationCallback searchCallback);
+
+    public MultiComparator getOriginalKeyComparator();
+
+    public void setOriginialKeyComparator(MultiComparator originalCmp);
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManagerFactory.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManagerFactory.java
new file mode 100644
index 0000000..d26e65f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManagerFactory.java
@@ -0,0 +1,5 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+public interface IFreePageManagerFactory {
+    public IFreePageManager createFreePageManager();
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IInMemoryFreePageManager.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IInMemoryFreePageManager.java
new file mode 100644
index 0000000..67935a7
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IInMemoryFreePageManager.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+public interface IInMemoryFreePageManager extends IFreePageManager {
+    public int getCapacity();
+
+    public void reset();
+
+    public boolean isFull();
+}
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
new file mode 100644
index 0000000..1557c75
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndex.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright 2009-2010 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.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+/**
+ * This interface describes the operations common to all indexes. Indexes
+ * implementing this interface can easily reuse existing index operators for
+ * dataflow. Users must perform operations on an via an {@link IIndexAccessor}.
+ * During dataflow, the lifecycle of IIndexes are handled through an {@link IIndexLifecycleManager}.
+ */
+public interface IIndex {
+
+    /**
+     * Initializes the persistent state of an index.
+     * An index cannot be created if it is in the activated state.
+     * Calling create on an index that is deactivated has the effect of clearing the index.
+     * 
+     * @throws HyracksDataException
+     *             if there is an error in the BufferCache while (un)pinning pages, (un)latching pages,
+     *             creating files, or deleting files
+     *             if the index is in the activated state
+     */
+    public void create() throws HyracksDataException;
+
+    /**
+     * Initializes the index's operational state. An index in the activated state may perform
+     * operations via an {@link IIndexAccessor}.
+     * 
+     * @throws HyracksDataException
+     *             if there is a problem in the BufferCache while (un)pinning pages, (un)latching pages,
+     *             creating files, or deleting files
+     */
+    public void activate() throws HyracksDataException;
+
+    /**
+     * Resets the operational state of the index. Calling clear has the same logical effect
+     * as calling deactivate(), destroy(), create(), then activate(), but not necessarily the
+     * same physical effect.
+     * 
+     * @throws HyracksDataException
+     *             if there is a problem in the BufferCache while (un)pinning pages, (un)latching pages,
+     *             creating files, or deleting files
+     *             if the index is not in the activated state
+     */
+    public void clear() throws HyracksDataException;
+
+    /**
+     * Deinitializes the index's operational state. An index in the deactivated state may not
+     * perform operations.
+     * 
+     * @throws HyracksDataException
+     *             if there is a problem in the BufferCache while (un)pinning pages, (un)latching pages,
+     *             creating files, or deleting files
+     */
+    public void deactivate() throws HyracksDataException;
+
+    /**
+     * Removes the persistent state of an index.
+     * An index cannot be destroyed if it is in the activated state.
+     * 
+     * @throws HyracksDataException
+     *             if there is an error in the BufferCache while (un)pinning pages, (un)latching pages,
+     *             creating files, or deleting files
+     *             if the index is already activated
+     */
+    public void destroy() throws HyracksDataException;
+
+    /**
+     * Creates an {@link IIndexAccessor} for performing operations on this index.
+     * An IIndexAccessor is not thread safe, but different IIndexAccessors can concurrently operate
+     * on the same {@link IIndex}.
+     * 
+     * @returns IIndexAccessor an accessor for this {@link IIndex}
+     * @param modificationCallback
+     *            the callback to be used for modification operations
+     * @param searchCallback
+     *            the callback to be used for search operations
+     */
+    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback);
+
+    /**
+     * Ensures that all pages (and tuples) of the index are logically consistent.
+     * An assertion error is thrown if validation fails.
+     * 
+     * @throws HyracksDataException
+     *             if there is an error performing validation
+     */
+    public void validate() throws HyracksDataException;
+
+    /**
+     * @return the {@link IBufferCache} underlying this index.
+     */
+    public IBufferCache getBufferCache();
+
+    /**
+     * @return the size, in bytes, of pre-allocated memory space that this index was allotted.
+     */
+    public long getMemoryAllocationSize();
+
+    /**
+     * @param fillFactor
+     * @param verifyInput
+     * @throws IndexException
+     */
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+            throws IndexException;
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexBulkLoadContext.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexBulkLoadContext.java
deleted file mode 100644
index a896d80..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexBulkLoadContext.java
+++ /dev/null
@@ -1,4 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.api;
-
-public interface IIndexBulkLoadContext {
-}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexBulkLoader.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexBulkLoader.java
new file mode 100644
index 0000000..86ed78f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexBulkLoader.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hyracks.storage.am.common.api;

+

+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;

+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;

+

+public interface IIndexBulkLoader {

+    /**

+     * Append a tuple to the index in the context of a bulk load.

+     * 

+     * @param tuple

+     *            Tuple to be inserted.

+     * @throws IndexException

+     *             If the input stream is invalid for bulk loading (e.g., is not sorted).

+     * @throws HyracksDataException

+     *             If the BufferCache throws while un/pinning or un/latching.

+     */

+    public void add(ITupleReference tuple) throws IndexException, HyracksDataException;

+

+    /**

+     * Finalize the bulk loading operation in the given context.

+     * 

+     * @throws IndexException

+     * @throws HyracksDataException

+     *             If the BufferCache throws while un/pinning or un/latching.

+     */

+    public void end() throws IndexException, HyracksDataException;

+

+}

diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexCursor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexCursor.java
index d29fd73..1c75da2 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexCursor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexCursor.java
@@ -19,16 +19,16 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
 public interface IIndexCursor {
-    public void open(ICursorInitialState initialState,
-            ISearchPredicate searchPred) throws HyracksDataException;      
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws IndexException,
+            HyracksDataException;
 
-    public boolean hasNext() throws HyracksDataException;
+    public boolean hasNext() throws HyracksDataException, IndexException;
 
     public void next() throws HyracksDataException;
 
     public void close() throws HyracksDataException;
 
-    public void reset();
+    public void reset() throws HyracksDataException, IndexException;
 
     public ITupleReference getTuple();
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexDataflowHelper.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexDataflowHelper.java
new file mode 100644
index 0000000..39b4553
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexDataflowHelper.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+
+public interface IIndexDataflowHelper {
+    public void create() throws HyracksDataException;
+
+    public void close() throws HyracksDataException;
+
+    public void open() throws HyracksDataException;
+
+    public void destroy() throws HyracksDataException;
+
+    public IIndex getIndexInstance();
+
+    public FileReference getFileReference();
+
+    public long getResourceID() throws HyracksDataException;
+    
+    public IHyracksTaskContext getTaskContext();
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManager.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManager.java
new file mode 100644
index 0000000..c4f43b9
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManager.java
@@ -0,0 +1,19 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IIndexLifecycleManager {
+    public IIndex getIndex(long resourceID);
+
+    public void register(long resourceID, IIndex index) throws HyracksDataException;
+
+    public void unregister(long resourceID) throws HyracksDataException;
+
+    public void open(long resourceID) throws HyracksDataException;
+
+    public void close(long resourceID);
+
+    public List<IIndex> getOpenIndexes();
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManagerProvider.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManagerProvider.java
new file mode 100644
index 0000000..7ae4a48
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexLifecycleManagerProvider.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public interface IIndexLifecycleManagerProvider extends Serializable {
+    public IIndexLifecycleManager getLifecycleManager(IHyracksTaskContext ctx);
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOpContext.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOpContext.java
deleted file mode 100644
index 7153f78..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOpContext.java
+++ /dev/null
@@ -1,8 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.api;
-
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-
-public interface IIndexOpContext {
-	void reset();
-	void reset(IndexOp newOp);
-}
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
new file mode 100644
index 0000000..c5bf83e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexOperationContext.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+
+public interface IIndexOperationContext {
+    void setOperation(IndexOperation newOp);
+    
+    IndexOperation getOperation();
+
+    void reset();
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IModificationOperationCallback.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IModificationOperationCallback.java
new file mode 100644
index 0000000..0b68ab0
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IModificationOperationCallback.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * This operation callback allows for arbitrary actions to be taken while traversing 
+ * an index structure. The {@link IModificationOperationCallback} will be called on 
+ * all modifying operations (e.g. insert, update, delete...) for all indexes.
+ * 
+ * @author zheilbron
+ */
+public interface IModificationOperationCallback {
+
+    /**
+     * This method is called on a tuple that is about to traverse an index's structure 
+     * (i.e. before any pages are pinned or latched). 
+     *
+     * The format of the tuple is the format that would be stored in the index itself.
+     * 
+     * @param tuple the tuple that is about to be operated on
+     */
+    public void before(ITupleReference tuple) throws HyracksDataException;
+
+    /**
+     * This method is called on a tuple when a tuple with a matching key is found for the 
+     * current operation. It is possible that tuple is null, in which case no tuple with a 
+     * matching key was found.
+     * 
+     * When found is called, the leaf page where the tuple resides will be pinned and latched, 
+     * so blocking operations should be avoided.
+     * 
+     * @param tuple a tuple with a matching key, otherwise null if none exists
+     */
+    public void found(ITupleReference before, ITupleReference after) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
new file mode 100644
index 0000000..65c9c8a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IModificationOperationCallbackFactory extends Serializable {
+    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource, IHyracksTaskContext ctx)
+            throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IOperationCallback.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IOperationCallback.java
deleted file mode 100644
index 9e66b43..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IOperationCallback.java
+++ /dev/null
@@ -1,9 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.api;
-
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-
-public interface IOperationCallback {
-    public void pre(ITupleReference tuple);
-
-    public void post(ITupleReference tuple);
-}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IOperationCallbackProvider.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IOperationCallbackProvider.java
deleted file mode 100644
index 974ef1a..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IOperationCallbackProvider.java
+++ /dev/null
@@ -1,7 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.api;
-
-import java.io.Serializable;
-
-public interface IOperationCallbackProvider extends Serializable {
-    public IOperationCallback getOperationCallback();
-}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISearchOperationCallback.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISearchOperationCallback.java
new file mode 100644
index 0000000..b62bbcb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISearchOperationCallback.java
@@ -0,0 +1,45 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * This operation callback allows for arbitrary actions to be taken while traversing 
+ * an index structure. The {@link ISearchOperationCallback} will be called on 
+ * all search operations for ordered indexes only.
+ * 
+ * @author zheilbron
+ */
+public interface ISearchOperationCallback {
+
+    /**
+     * During an index search operation, this method will be called on tuples as they are 
+     * passed by with a search cursor. This call will be invoked while a leaf page is latched 
+     * and pinned. If the call returns false, then the page will be unlatched and unpinned and 
+     * {@link #reconcile(ITupleReference)} will be called with the tuple that was not proceeded 
+     * on.
+     * 
+     * @param tuple the tuple that is being passed over by the search cursor
+     * @return true to proceed otherwise false to unlatch and unpin, leading to reconciliation
+     */
+    public boolean proceed(ITupleReference tuple) throws HyracksDataException;
+
+    /**
+     * This method is only called on a tuple that was not 'proceeded' on 
+     * (see {@link #proceed(ITupleReference)}). This method allows an opportunity to reconcile 
+     * by performing any necessary actions before resuming the search (e.g. a try-lock may have 
+     * failed in the proceed call, and now in reconcile we should take a full (blocking) lock).
+     * 
+     * @param tuple the tuple that failed to proceed
+     */
+    public void reconcile(ITupleReference tuple) throws HyracksDataException;
+
+    /**
+     * This method is only called on a tuple that was reconciled on, but not found after 
+     * retraversing. This method allows an opportunity to cancel some action that was taken in 
+     * {@link #reconcile(ITupleReference))}.
+     * 
+     * @param tuple the tuple that was previously reconciled
+     */
+    public void cancel(ITupleReference tuple) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java
new file mode 100644
index 0000000..6389b8d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISearchOperationCallbackFactory extends Serializable {
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+            throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISlotManager.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISlotManager.java
index 2619493..3e5e018 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISlotManager.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISlotManager.java
@@ -21,27 +21,26 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
 public interface ISlotManager {
-	public int findTupleIndex(ITupleReference searchKey,
-			ITreeIndexTupleReference frameTuple, MultiComparator multiCmp,
-			FindTupleMode mode, FindTupleNoExactMatchPolicy matchPolicy);
+    public int findTupleIndex(ITupleReference searchKey, ITreeIndexTupleReference frameTuple, MultiComparator multiCmp,
+            FindTupleMode mode, FindTupleNoExactMatchPolicy matchPolicy);
 
-	public int getGreatestKeyIndicator();
-	
-	public int getErrorIndicator();
+    public int getGreatestKeyIndicator();
 
-	public void setFrame(ITreeIndexFrame frame);
-	
-	public int insertSlot(int tupleIndex, int tupleOff);
+    public int getErrorIndicator();
 
-	public int getSlotStartOff();
+    public void setFrame(ITreeIndexFrame frame);
 
-	public int getSlotEndOff();
+    public int getTupleOff(int slotOff);
 
-	public int getTupleOff(int slotOff);
+    public int insertSlot(int tupleIndex, int tupleOff);
 
-	public void setSlot(int slotOff, int value);
+    public int getSlotStartOff();
 
-	public int getSlotOff(int tupleIndex);
+    public int getSlotEndOff();
 
-	public int getSlotSize();
+    public int getSlotOff(int tupleIndex);
+
+    public int getSlotSize();
+
+    public void setSlot(int slotOff, int value);
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
index 52626cf..2313c2e 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
@@ -16,7 +16,6 @@
 package edu.uci.ics.hyracks.storage.am.common.api;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
 
 /**
  * Interface describing the operations of tree-based index structures. Indexes
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
index c33a8d8..612af25 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
@@ -20,20 +20,21 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 
 public interface ITreeIndexFrame {
 
-	public void initBuffer(byte level);
-	
+    public void initBuffer(byte level);
+
     public FrameOpSpaceStatus hasSpaceInsert(ITupleReference tuple);
-	
-	public void insert(ITupleReference tuple, int tupleIndex);    
-    
-	public FrameOpSpaceStatus hasSpaceUpdate(ITupleReference newTuple, int oldTupleIndex);
-	
-	public void update(ITupleReference newTuple, int oldTupleIndex, boolean inPlace);    
-    
+
+    public void insert(ITupleReference tuple, int tupleIndex);
+
+    public FrameOpSpaceStatus hasSpaceUpdate(ITupleReference newTuple, int oldTupleIndex);
+
+    public void update(ITupleReference newTuple, int oldTupleIndex, boolean inPlace);
+
     public void delete(ITupleReference tuple, int tupleIndex);
 
     // returns true if slots were modified, false otherwise
@@ -57,11 +58,11 @@
     public ICachedPage getPage();
 
     public ByteBuffer getBuffer();
-    
+
     // for debugging
     public String printHeader();
 
-    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) throws TreeIndexException;
+    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey);
 
     public ISlotManager getSlotManager();
 
@@ -87,6 +88,9 @@
     public ITreeIndexTupleWriter getTupleWriter();
 
     public int getPageHeaderSize();
-    
+
     public ITreeIndexTupleReference createTupleReference();
+
+    public void setMultiComparator(MultiComparator cmp);
+
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrame.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrame.java
index 9e95970..b48ded4 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrame.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrame.java
@@ -47,4 +47,9 @@
 	
 	// Set special validity flag.
 	public void setValid(boolean isValid);
+	
+	// Special placeholder for LSN information. Used for transactional LSM indexes.
+	public long getLSN();
+	
+	public void setLSN(long lsn);
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexTupleWriter.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexTupleWriter.java
index 30e8f39..43991f1 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexTupleWriter.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexTupleWriter.java
@@ -21,19 +21,22 @@
 
 public interface ITreeIndexTupleWriter {
     public int writeTuple(ITupleReference tuple, ByteBuffer targetBuf, int targetOff);
-    
+
     public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff);
 
     public int bytesRequired(ITupleReference tuple);
 
-    public int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf,
-            int targetOff);
+    public int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf, int targetOff);
 
     public int bytesRequired(ITupleReference tuple, int startField, int numFields);
 
     // return a tuplereference instance that can read the tuple written by this
-    // writer
-    // the main idea is that the format of the written tuple may not be the same
+    // writer the main idea is that the format of the written tuple may not be the same
     // as the format written by this writer
     public ITreeIndexTupleReference createTupleReference();
+
+    // This method is only used by the BTree leaf frame split method since tuples
+    // in the LSM-BTree can be either matter or anti-matter tuples and we want
+    // to calculate the size of all tuples in the frame.
+    public int getCopySpaceRequired(ITupleReference tuple);
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IndexType.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IndexType.java
deleted file mode 100644
index 6f83e0b..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IndexType.java
+++ /dev/null
@@ -1,5 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.api;
-
-public enum IndexType {
-	BTREE, RTREE, INVERTED
-}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/UnsortedInputException.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/UnsortedInputException.java
new file mode 100644
index 0000000..e3304c9
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/UnsortedInputException.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2010 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.common.api;
+
+/**
+ * Thrown when trying to bulk load an index with an unsorted input stream. 
+ */
+public class UnsortedInputException extends IndexException {
+	private static final long serialVersionUID = 1L;
+	
+	public UnsortedInputException(Exception e) {
+		super(e);
+	}
+	
+	public UnsortedInputException(String message) {
+        super(message);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java
new file mode 100644
index 0000000..c26bbfa
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractIndexOperatorDescriptor.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2009-2012 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.common.dataflow;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+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.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+
+public abstract class AbstractIndexOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor implements
+        IIndexOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final IFileSplitProvider fileSplitProvider;
+    protected final IStorageManagerInterface storageManager;
+    protected final IIndexLifecycleManagerProvider lifecycleManagerProvider;
+    protected final IIndexDataflowHelperFactory dataflowHelperFactory;
+    protected final ITupleFilterFactory tupleFilterFactory;
+    protected final boolean retainInput;
+    protected final ISearchOperationCallbackFactory searchOpCallbackFactory;
+    protected final IModificationOperationCallbackFactory modificationOpCallbackFactory;
+    protected final ILocalResourceFactoryProvider localResourceFactoryProvider;
+
+    public AbstractIndexOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity, int outputArity,
+            RecordDescriptor recDesc, IStorageManagerInterface storageManager,
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
+            IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
+            boolean retainInput, ILocalResourceFactoryProvider localResourceFactoryProvider,
+            ISearchOperationCallbackFactory searchOpCallbackFactory,
+            IModificationOperationCallbackFactory modificationOpCallbackFactory) {
+        super(spec, inputArity, outputArity);
+        this.fileSplitProvider = fileSplitProvider;
+        this.storageManager = storageManager;
+        this.lifecycleManagerProvider = lifecycleManagerProvider;
+        this.dataflowHelperFactory = dataflowHelperFactory;
+        this.retainInput = retainInput;
+        this.tupleFilterFactory = tupleFilterFactory;
+        this.localResourceFactoryProvider = localResourceFactoryProvider;
+        this.searchOpCallbackFactory = searchOpCallbackFactory;
+        this.modificationOpCallbackFactory = modificationOpCallbackFactory;
+        if (outputArity > 0) {
+            recordDescriptors[0] = recDesc;
+        }
+    }
+
+    @Override
+    public IFileSplitProvider getFileSplitProvider() {
+        return fileSplitProvider;
+    }
+
+    @Override
+    public IStorageManagerInterface getStorageManager() {
+        return storageManager;
+    }
+
+    @Override
+    public IIndexLifecycleManagerProvider getLifecycleManagerProvider() {
+        return lifecycleManagerProvider;
+    }
+
+    @Override
+    public RecordDescriptor getRecordDescriptor() {
+        return recordDescriptors[0];
+    }
+
+    @Override
+    public IIndexDataflowHelperFactory getIndexDataflowHelperFactory() {
+        return dataflowHelperFactory;
+    }
+
+    @Override
+    public boolean getRetainInput() {
+        return retainInput;
+    }
+
+    @Override
+    public ISearchOperationCallbackFactory getSearchOpCallbackFactory() {
+        return searchOpCallbackFactory;
+    }
+    
+    @Override
+    public IModificationOperationCallbackFactory getModificationOpCallbackFactory() {
+        return modificationOpCallbackFactory;
+    }
+
+    @Override
+    public ITupleFilterFactory getTupleFilterFactory() {
+        return tupleFilterFactory;
+    }
+    
+    @Override
+    public ILocalResourceFactoryProvider getLocalResourceFactoryProvider() {
+        return localResourceFactoryProvider;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
index 9f0fbc9..ffcf7a1 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2009-2010 by The Regents of the University of California
+ * Copyright 2009-2012 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
@@ -19,104 +19,57 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
-public abstract class AbstractTreeIndexOperatorDescriptor extends
-		AbstractSingleActivityOperatorDescriptor implements
-		ITreeIndexOperatorDescriptor {
+public abstract class AbstractTreeIndexOperatorDescriptor extends AbstractIndexOperatorDescriptor {
 
-	private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1L;
 
-	protected final IFileSplitProvider fileSplitProvider;
+    protected final ITypeTraits[] typeTraits;
+    protected final IBinaryComparatorFactory[] comparatorFactories;
+    protected final int[] bloomFilterKeyFields;
 
-	protected final IBinaryComparatorFactory[] comparatorFactories;
+    public AbstractTreeIndexOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity, int outputArity,
+            RecordDescriptor recDesc, IStorageManagerInterface storageManager,
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
+            IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
+            boolean retainInput, ILocalResourceFactoryProvider localResourceFactoryProvider,
+            ISearchOperationCallbackFactory searchOpCallbackFactory,
+            IModificationOperationCallbackFactory modificationOpCallbackFactory) {
+        super(spec, inputArity, outputArity, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider,
+                dataflowHelperFactory, tupleFilterFactory, retainInput, localResourceFactoryProvider,
+                searchOpCallbackFactory, modificationOpCallbackFactory);
+        this.typeTraits = typeTraits;
+        this.comparatorFactories = comparatorFactories;
+        this.bloomFilterKeyFields = bloomFilterKeyFields;
+    }
 
-	protected final IStorageManagerInterface storageManager;
-	protected final IIndexRegistryProvider<IIndex> indexRegistryProvider;
+    public IBinaryComparatorFactory[] getTreeIndexComparatorFactories() {
+        return comparatorFactories;
+    }
 
-	protected final ITypeTraits[] typeTraits;
-	protected final IIndexDataflowHelperFactory dataflowHelperFactory;
-	protected final ITupleFilterFactory tupleFilterFactory;
-	
-	protected final boolean retainInput;
-    protected final IOperationCallbackProvider opCallbackProvider;
+    public ITypeTraits[] getTreeIndexTypeTraits() {
+        return typeTraits;
+    }
 
-	public AbstractTreeIndexOperatorDescriptor(IOperatorDescriptorRegistry spec,
-			int inputArity, int outputArity, RecordDescriptor recDesc,
-			IStorageManagerInterface storageManager,
-			IIndexRegistryProvider<IIndex> indexRegistryProvider,
-			IFileSplitProvider fileSplitProvider,
-			ITypeTraits[] typeTraits,
-			IBinaryComparatorFactory[] comparatorFactories,
-			IIndexDataflowHelperFactory dataflowHelperFactory,
-			ITupleFilterFactory tupleFilterFactory,
-			boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
-		super(spec, inputArity, outputArity);
-		this.fileSplitProvider = fileSplitProvider;
-		this.storageManager = storageManager;
-		this.indexRegistryProvider = indexRegistryProvider;
-		this.typeTraits = typeTraits;
-		this.comparatorFactories = comparatorFactories;
-		this.dataflowHelperFactory = dataflowHelperFactory;
-		this.retainInput = retainInput;
-		this.tupleFilterFactory = tupleFilterFactory;
-        this.opCallbackProvider = opCallbackProvider;
-		if (outputArity > 0) {
-			recordDescriptors[0] = recDesc;
-		}
-	}
+    public int[] getTreeIndexBloomFilterKeyFields() {
+        return bloomFilterKeyFields;
+    }
 
-	@Override
-	public IFileSplitProvider getFileSplitProvider() {
-		return fileSplitProvider;
-	}
+    @Override
+    public boolean getRetainInput() {
+        return retainInput;
+    }
 
-	@Override
-	public IBinaryComparatorFactory[] getTreeIndexComparatorFactories() {
-		return comparatorFactories;
-	}
-
-	@Override
-	public ITypeTraits[] getTreeIndexTypeTraits() {
-		return typeTraits;
-	}
-
-	@Override
-	public IStorageManagerInterface getStorageManager() {
-		return storageManager;
-	}
-
-	@Override
-	public IIndexRegistryProvider<IIndex> getIndexRegistryProvider() {
-		return indexRegistryProvider;
-	}
-
-	@Override
-	public RecordDescriptor getRecordDescriptor() {
-		return recordDescriptors[0];
-	}
-
-	@Override
-	public IIndexDataflowHelperFactory getIndexDataflowHelperFactory() {
-		return dataflowHelperFactory;
-	}
-	
-	@Override
-	public boolean getRetainInput() {
-		return retainInput;
-	}
-	
-	@Override
-	public IOperationCallbackProvider getOpCallbackProvider() {
-	    return opCallbackProvider;
-	}
-	
-	@Override
-	public ITupleFilterFactory getTupleFilterFactory() {
-		return tupleFilterFactory;
-	}
+    @Override
+    public ITupleFilterFactory getTupleFilterFactory() {
+        return tupleFilterFactory;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndex.java
deleted file mode 100644
index 64cbd58..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndex.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
-import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.common.api.IndexType;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-
-/**
- * Interface describing the operations common to all index structures. Indexes
- * implementing this interface can easily reuse existing index operators for
- * dataflow. Users must perform operations on an IIndex via an IIndexAccessor.
- */
-public interface IIndex {
-    /**
-     * Initializes the persistent state of an index, e.g., the root page, and
-     * metadata pages.
-     * 
-     * @param indexFileId
-     *            The file id to use for this index.
-     * @throws HyracksDataException
-     *             If the BufferCache throws while un/pinning or un/latching.
-     */
-    public void create(int indexFileId) throws HyracksDataException;
-
-    /**
-     * Opens the index backed by the given file id.
-     * 
-     * @param indexFileId
-     *            The file id backing this index.
-     */
-    public void open(int indexFileId) throws HyracksDataException;
-
-    /**
-     * Closes the index.
-     */
-    public void close() throws HyracksDataException;
-
-    /**
-     * Creates an index accessor for performing operations on this index.
-     * (insert/delete/update/search/diskorderscan). An IIndexAccessor is not
-     * thread safe, but different IIndexAccessors can concurrently operate
-     * on the same IIndex
-     * 
-     * @returns IIndexAccessor An accessor for this tree.
-     */
-    public IIndexAccessor createAccessor();
-
-    /**
-     * Prepares the index for bulk loading, returning a bulk load context. The
-     * index may require to be empty for bulk loading.
-     * 
-     * @param fillFactor
-     *            Desired fill factor in [0, 1.0].
-     * @throws HyracksDataException
-     *             If the BufferCache throws while un/pinning or un/latching.
-     * @throws IndexException
-     *             For example, if the index was already loaded and only
-     *             supports a single load.
-     * @returns A new context for bulk loading, required for appending tuples.
-     */
-    public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws IndexException, HyracksDataException;
-
-    /**
-     * Append a tuple to the index in the context of a bulk load.
-     * 
-     * @param tuple
-     *            Tuple to be inserted.
-     * @param ictx
-     *            Existing bulk load context.
-     * @throws HyracksDataException
-     *             If the BufferCache throws while un/pinning or un/latching.
-     */
-    public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException;
-
-    /**
-     * Finalize the bulk loading operation in the given context.
-     * 
-     * @param ictx
-     *            Existing bulk load context to be finalized.
-     * @throws HyracksDataException
-     *             If the BufferCache throws while un/pinning or un/latching.
-     */
-    public void endBulkLoad(IIndexBulkLoadContext ictx) throws HyracksDataException;
-
-    /**
-     * @return BufferCache underlying this index.
-     */
-    public IBufferCache getBufferCache();
-
-    /**
-     * @return An enum of the concrete type of this index.
-     */
-    public IndexType getIndexType();
-}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexDataflowHelperFactory.java
index ddca470..71760c9 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexDataflowHelperFactory.java
@@ -18,8 +18,9 @@
 import java.io.Serializable;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
 
 public interface IIndexDataflowHelperFactory extends Serializable {
-    public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc,
+    public IIndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc,
             final IHyracksTaskContext ctx, int partition);
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
index e37d374..6193414 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
@@ -18,21 +18,31 @@
 import edu.uci.ics.hyracks.api.dataflow.IActivity;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 public interface IIndexOperatorDescriptor extends IActivity {
     public IFileSplitProvider getFileSplitProvider();
 
     public IStorageManagerInterface getStorageManager();
 
-    public IIndexRegistryProvider<IIndex> getIndexRegistryProvider();    
-    
+    public IIndexLifecycleManagerProvider getLifecycleManagerProvider();
+
     public RecordDescriptor getRecordDescriptor();
-    
+
     public IIndexDataflowHelperFactory getIndexDataflowHelperFactory();
-    
+
     public boolean getRetainInput();
+
+    public ISearchOperationCallbackFactory getSearchOpCallbackFactory();
     
-    public IOperationCallbackProvider getOpCallbackProvider();
+    public IModificationOperationCallbackFactory getModificationOpCallbackFactory();
+    
+    public ITupleFilterFactory getTupleFilterFactory();
+    
+    public ILocalResourceFactoryProvider getLocalResourceFactoryProvider();
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java
deleted file mode 100644
index ed20de0..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow;
-
-import java.io.Serializable;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-
-public interface IIndexRegistryProvider<IndexType> extends Serializable {
-	public IndexRegistry<IndexType> getRegistry(IHyracksTaskContext ctx);
-}
\ No newline at end of file
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
new file mode 100644
index 0000000..1b6271d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2012 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.common.dataflow;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
+
+public class IndexBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
+    private final IIndexOperatorDescriptor opDesc;
+    private final IHyracksTaskContext ctx;
+    private final float fillFactor;
+    private final boolean verifyInput;
+    private final long numElementsHint;
+    private final IIndexDataflowHelper indexHelper;
+    private FrameTupleAccessor accessor;
+    private IIndex index;
+    private IIndexBulkLoader bulkLoader;
+    private IRecordDescriptorProvider recDescProvider;
+    private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
+
+    public IndexBulkLoadOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            int[] fieldPermutation, float fillFactor, boolean verifyInput, long numElementsHint,
+            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.recDescProvider = recordDescProvider;
+        tuple.setFieldPermutation(fieldPermutation);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        RecordDescriptor recDesc = recDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
+        accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+        indexHelper.open();
+        index = indexHelper.getIndexInstance();
+        try {
+            bulkLoader = index.createBulkLoader(fillFactor, verifyInput, numElementsHint);
+        } catch (Exception e) {
+            indexHelper.close();
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            tuple.reset(accessor, i);
+            try {
+                bulkLoader.add(tuple);
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        try {
+            bulkLoader.end();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        } finally {
+            indexHelper.close();
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexCreateOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexCreateOperatorNodePushable.java
new file mode 100644
index 0000000..f965f01
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexCreateOperatorNodePushable.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2012 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.common.dataflow;
+
+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.std.base.AbstractOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
+
+public class IndexCreateOperatorNodePushable extends AbstractOperatorNodePushable {
+    private final IIndexDataflowHelper indexHelper;
+
+    public IndexCreateOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition) {
+        this.indexHelper = opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(opDesc, ctx, partition);
+    }
+
+    @Override
+    public void deinitialize() throws HyracksDataException {
+    }
+
+    @Override
+    public int getInputArity() {
+        return 0;
+    }
+
+    @Override
+    public IFrameWriter getInputFrameWriter(int index) {
+        return null;
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        indexHelper.create();
+    }
+
+    @Override
+    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+    }
+}
\ No newline at end of file
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 fa95ce4..e46efff 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
@@ -15,105 +15,132 @@
 
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
+import java.io.IOException;
+
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 
-public abstract class IndexDataflowHelper {
-    protected IIndex index;
-    protected int indexFileId = -1;
+public abstract class IndexDataflowHelper implements IIndexDataflowHelper {
 
-    protected final int partition;
     protected final IIndexOperatorDescriptor opDesc;
     protected final IHyracksTaskContext ctx;
+    protected final IIndexLifecycleManager lcManager;
+    protected final ILocalResourceRepository localResourceRepository;
+    protected final ResourceIdFactory resourceIdFactory;
+    protected final FileReference file;
+    protected final int partition;
+
+    protected IIndex index;
 
     public IndexDataflowHelper(IIndexOperatorDescriptor opDesc, final IHyracksTaskContext ctx, int partition) {
         this.opDesc = opDesc;
         this.ctx = ctx;
+        this.lcManager = opDesc.getLifecycleManagerProvider().getLifecycleManager(ctx);
+        this.localResourceRepository = opDesc.getStorageManager().getLocalResourceRepository(ctx);
+        this.resourceIdFactory = opDesc.getStorageManager().getResourceIdFactory(ctx);
         this.partition = partition;
+        this.file = opDesc.getFileSplitProvider().getFileSplits()[partition].getLocalFile();
     }
 
-    public void init(boolean forceCreate) throws HyracksDataException {
-        IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
-        IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
-        IndexRegistry<IIndex> indexRegistry = opDesc.getIndexRegistryProvider().getRegistry(ctx);
-        FileReference fileRef = getFilereference();
-        int fileId = -1;
-        boolean fileIsMapped = false;
-        synchronized (fileMapProvider) {
-            fileIsMapped = fileMapProvider.isMapped(fileRef);
-            if (!fileIsMapped) {
-                bufferCache.createFile(fileRef);
-            }
-            fileId = fileMapProvider.lookupFileId(fileRef);
-            try {
-                // Also creates the file if it doesn't exist yet.
-                bufferCache.openFile(fileId);
-            } catch (HyracksDataException e) {
-                // Revert state of buffer cache since file failed to open.
-                if (!fileIsMapped) {
-                    bufferCache.deleteFile(fileId, false);
-                }
-                throw e;
-            }
-        }
-        // Only set indexFileId member after openFile() succeeds.
-        indexFileId = fileId;
-        // Create new index instance and register it.
-        synchronized (indexRegistry) {
-            // Check if the index has already been registered.
-            boolean register = false;
-            index = indexRegistry.get(indexFileId);
-            if (index == null) {
-                index = createIndexInstance();
-                register = true;
-            }
-            if (forceCreate) {
-                index.create(indexFileId);
-            }
-            index.open(indexFileId);
-            if (register) {
-                indexRegistry.register(indexFileId, index);
-            }
-        }
-    }
+    protected abstract IIndex createIndexInstance() throws HyracksDataException;
 
-    public abstract IIndex createIndexInstance() throws HyracksDataException;
-
-    public FileReference getFilereference() {
-        IFileSplitProvider fileSplitProvider = opDesc.getFileSplitProvider();
-        return fileSplitProvider.getFileSplits()[partition].getLocalFile();
-    }
-
-    public void deinit() throws HyracksDataException {
-        if (indexFileId != -1) {
-            IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
-            bufferCache.closeFile(indexFileId);
-            indexFileId = -1;
-        }
-    }
-
-    public IIndex getIndex() {
+    public IIndex getIndexInstance() {
         return index;
     }
 
-    public IHyracksTaskContext getHyracksTaskContext() {
+    public void create() throws HyracksDataException {
+        synchronized (lcManager) {
+            long resourceID = getResourceID();
+            index = lcManager.getIndex(resourceID);
+            if (index != null) {
+                lcManager.unregister(resourceID);
+            } else {
+                index = createIndexInstance();
+            }
+
+            // The previous resource ID needs to be removed since calling IIndex.create() may possibly destroy 
+            // 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());
+            }
+            index.create();
+            try {
+                //TODO Create LocalResource through LocalResourceFactory interface
+                resourceID = resourceIdFactory.createId();
+                ILocalResourceFactory localResourceFactory = opDesc.getLocalResourceFactoryProvider()
+                        .getLocalResourceFactory();
+                localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
+                        .getPath(), partition));
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+            lcManager.register(resourceID, index);
+        }
+    }
+
+    public void open() throws HyracksDataException {
+        synchronized (lcManager) {
+            long resourceID = getResourceID();
+
+            if (resourceID == -1) {
+                throw new HyracksDataException("Index does not have a valid resource ID. Has it been created yet?");
+            }
+
+            index = lcManager.getIndex(resourceID);
+            if (index == null) {
+                index = createIndexInstance();
+                lcManager.register(resourceID, index);
+            }
+            lcManager.open(resourceID);
+        }
+    }
+
+    public void close() throws HyracksDataException {
+        synchronized (lcManager) {
+            lcManager.close(getResourceID());
+        }
+    }
+
+    public void destroy() throws HyracksDataException {
+        synchronized (lcManager) {
+            long resourceID = getResourceID();
+            index = lcManager.getIndex(resourceID);
+            if (index != null) {
+                lcManager.unregister(resourceID);
+            } else {
+                index = createIndexInstance();
+            }
+
+            if (resourceID != -1) {
+                localResourceRepository.deleteResourceByName(file.getFile().getPath());
+            }
+            index.destroy();
+        }
+    }
+
+    public FileReference getFileReference() {
+        return file;
+    }
+
+    public long getResourceID() throws HyracksDataException {
+        LocalResource localResource = localResourceRepository.getResourceByName(file.getFile().getPath());
+        if (localResource == null) {
+            return -1;
+        } else {
+            return localResource.getResourceId();
+        }
+    }
+
+    public IHyracksTaskContext getTaskContext() {
         return ctx;
     }
-
-    public IIndexOperatorDescriptor getOperatorDescriptor() {
-        return opDesc;
-    }
-
-    public int getIndexFileId() {
-        return indexFileId;
-    }
-
-    public IOperationCallbackProvider getOpCallbackProvider() {
-        return opDesc.getOpCallbackProvider();
-    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java
new file mode 100644
index 0000000..6f890d7
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2009-2010 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.common.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+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.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
+
+public class IndexDropOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public IndexDropOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManagerInterface storageManager,
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
+            IIndexDataflowHelperFactory dataflowHelperFactory) {
+        // TODO: providing the type traits below is a hack to allow:
+        // 1) Type traits not to be specified when creating the drop operator
+        // 2) The LSMRTreeDataflowHelper to get acceptable type traits
+        // This should eventually not be *hacked*, but I don't know the proper fix yet. -zheilbron
+        super(spec, 0, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, new ITypeTraits[] {
+                IntegerPointable.TYPE_TRAITS, IntegerPointable.TYPE_TRAITS }, new IBinaryComparatorFactory[] { null }, null,
+                dataflowHelperFactory, null, false, NoOpLocalResourceFactoryProvider.INSTANCE,
+                NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new IndexDropOperatorNodePushable(this, ctx, partition);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
new file mode 100644
index 0000000..73bed72
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2009-2012 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.common.dataflow;
+
+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.std.base.AbstractOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
+
+public class IndexDropOperatorNodePushable extends AbstractOperatorNodePushable {
+    private final IIndexDataflowHelper indexHelper;
+
+    public IndexDropOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition) {
+        this.indexHelper = opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(opDesc, ctx, partition);
+    }
+
+    @Override
+    public void deinitialize() throws HyracksDataException {
+    }
+
+    @Override
+    public int getInputArity() {
+        return 0;
+    }
+
+    @Override
+    public IFrameWriter getInputFrameWriter(int index) {
+        return null;
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        indexHelper.destroy();
+    }
+
+    @Override
+    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
new file mode 100644
index 0000000..059f42a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
@@ -0,0 +1,144 @@
+/*
+ * Copyright 2009-2012 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.common.dataflow;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+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.IIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilter;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
+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.tuples.PermutingFrameTupleReference;
+
+public class IndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+    protected final IIndexOperatorDescriptor opDesc;
+    protected final IHyracksTaskContext ctx;
+    protected final IIndexDataflowHelper indexHelper;
+    protected final IRecordDescriptorProvider recordDescProvider;
+    protected final IndexOperation op;
+    protected final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
+    protected FrameTupleAccessor accessor;
+    protected FrameTupleReference frameTuple;
+    protected ByteBuffer writeBuffer;
+    protected IIndexAccessor indexAccessor;
+    protected ITupleFilter tupleFilter;
+    protected IModificationOperationCallback modCallback;
+
+    public IndexInsertUpdateDeleteOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider, IndexOperation op) {
+        this.opDesc = opDesc;
+        this.ctx = ctx;
+        this.indexHelper = opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(opDesc, ctx, partition);
+        this.recordDescProvider = recordDescProvider;
+        this.op = op;
+        tuple.setFieldPermutation(fieldPermutation);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
+        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+        writeBuffer = ctx.allocateFrame();
+        writer.open();
+        indexHelper.open();
+        IIndex index = indexHelper.getIndexInstance();
+        try {
+            modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
+                    indexHelper.getResourceID(), index, ctx);
+            indexAccessor = index.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+            ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
+            if (tupleFilterFactory != null) {
+                tupleFilter = tupleFilterFactory.createTupleFilter(indexHelper.getTaskContext());
+                frameTuple = new FrameTupleReference();
+            }
+        } catch (Exception e) {
+            indexHelper.close();
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            try {
+                if (tupleFilter != null) {
+                    frameTuple.reset(accessor, i);
+                    if (!tupleFilter.accept(frameTuple)) {
+                        continue;
+                    }
+                }
+                tuple.reset(accessor, i);
+
+                switch (op) {
+                    case INSERT: {
+                        indexAccessor.insert(tuple);
+                        break;
+                    }
+                    case UPDATE: {
+                        indexAccessor.update(tuple);
+                        break;
+                    }
+                    case UPSERT: {
+                        indexAccessor.upsert(tuple);
+                        break;
+                    }
+                    case DELETE: {
+                        indexAccessor.delete(tuple);
+                        break;
+                    }
+                    default: {
+                        throw new HyracksDataException("Unsupported operation " + op
+                                + " in tree index InsertUpdateDelete operator");
+                    }
+                }
+            } catch (HyracksDataException e) {
+                throw e;
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        // Pass a copy of the frame to next op.
+        System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
+        FrameUtils.flushFrame(writeBuffer, writer);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        try {
+            writer.close();
+        } finally {
+            indexHelper.close();
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        writer.fail();
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java
new file mode 100644
index 0000000..197aecc
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexLifecycleManager.java
@@ -0,0 +1,181 @@
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
+
+import java.util.ArrayList;
+import java.util.Collections;
+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.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+
+public class IndexLifecycleManager implements IIndexLifecycleManager {
+    private static final long DEFAULT_MEMORY_BUDGET = 1024 * 1024 * 100; // 100 megabytes
+
+    private final Map<Long, IndexInfo> indexInfos;
+    private final long memoryBudget;
+
+    private long memoryUsed;
+
+    public IndexLifecycleManager() {
+        this(DEFAULT_MEMORY_BUDGET);
+    }
+
+    public IndexLifecycleManager(long memoryBudget) {
+        this.indexInfos = new HashMap<Long, IndexInfo>();
+        this.memoryBudget = memoryBudget;
+        this.memoryUsed = 0;
+    }
+
+    private boolean evictCandidateIndex() throws HyracksDataException {
+        // Why min()? As a heuristic for eviction, we will take an open index (an index consuming memory) 
+        // that is not being used (refcount == 0) and has been least recently used. The sort order defined 
+        // for IndexInfo maintains this. See IndexInfo.compareTo().
+        IndexInfo info = Collections.min(indexInfos.values());
+        if (info.referenceCount != 0 || !info.isOpen) {
+            return false;
+        }
+
+        info.index.deactivate();
+        memoryUsed -= info.index.getMemoryAllocationSize();
+        info.isOpen = false;
+
+        return true;
+    }
+
+    @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();
+            memoryUsed -= info.index.getMemoryAllocationSize();
+        }
+    }
+
+    @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.");
+        }
+
+        long inMemorySize = info.index.getMemoryAllocationSize();
+        while (memoryUsed + inMemorySize > memoryBudget) {
+            if (!evictCandidateIndex()) {
+                throw new HyracksDataException("Cannot activate index since memory budget would be exceeded.");
+            }
+        }
+
+        if (!info.isOpen) {
+            info.index.activate();
+            info.isOpen = true;
+            memoryUsed += inMemorySize;
+        }
+        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;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
new file mode 100644
index 0000000..98beea2
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -0,0 +1,165 @@
+/*
+ * Copyright 2009-2010 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.common.dataflow;
+
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+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.common.api.IIndex;
+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.IIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+
+public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+    protected final IIndexOperatorDescriptor opDesc;
+    protected final IHyracksTaskContext ctx;
+    protected final IIndexDataflowHelper indexHelper;
+    protected FrameTupleAccessor accessor;
+
+    protected ByteBuffer writeBuffer;
+    protected FrameTupleAppender appender;
+    protected ArrayTupleBuilder tb;
+    protected DataOutput dos;
+
+    protected IIndex index;
+    protected ISearchPredicate searchPred;
+    protected IIndexCursor cursor;
+    protected IIndexAccessor indexAccessor;
+
+    protected final RecordDescriptor inputRecDesc;
+    protected final boolean retainInput;
+    protected FrameTupleReference frameTuple;
+
+    public IndexSearchOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            IRecordDescriptorProvider recordDescProvider) {
+        this.opDesc = opDesc;
+        this.ctx = ctx;
+        this.indexHelper = opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(opDesc, ctx, partition);
+        this.retainInput = opDesc.getRetainInput();
+        this.inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
+    }
+
+    protected abstract ISearchPredicate createSearchPredicate();
+
+    protected abstract void resetSearchPredicate(int tupleIndex);
+
+    protected IIndexCursor createCursor() {
+        return indexAccessor.createSearchCursor();
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+        writer.open();
+        indexHelper.open();
+        index = indexHelper.getIndexInstance();
+        try {
+            searchPred = createSearchPredicate();
+            writeBuffer = ctx.allocateFrame();
+            tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
+            dos = tb.getDataOutput();
+            appender = new FrameTupleAppender(ctx.getFrameSize());
+            appender.reset(writeBuffer, true);
+            ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
+                    .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
+            indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, searchCallback);
+            cursor = createCursor();
+            if (retainInput) {
+                frameTuple = new FrameTupleReference();
+            }
+        } catch (Exception e) {
+            indexHelper.close();
+            throw new HyracksDataException(e);
+        }
+    }
+
+    protected void writeSearchResults(int tupleIndex) throws Exception {
+        while (cursor.hasNext()) {
+            tb.reset();
+            cursor.next();
+            if (retainInput) {
+                frameTuple.reset(accessor, tupleIndex);
+                for (int i = 0; i < frameTuple.getFieldCount(); i++) {
+                    dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
+                    tb.addFieldEndOffset();
+                }
+            }
+            ITupleReference tuple = cursor.getTuple();
+            for (int i = 0; i < tuple.getFieldCount(); i++) {
+                dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+                tb.addFieldEndOffset();
+            }
+            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                FrameUtils.flushFrame(writeBuffer, writer);
+                appender.reset(writeBuffer, true);
+                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                    throw new IllegalStateException();
+                }
+            }
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        try {
+            for (int i = 0; i < tupleCount; i++) {
+                resetSearchPredicate(i);
+                cursor.reset();
+                indexAccessor.search(cursor, searchPred);
+                writeSearchResults(i);
+            }
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        try {
+            if (appender.getTupleCount() > 0) {
+                FrameUtils.flushFrame(writeBuffer, writer);
+            }
+            writer.close();
+            try {
+                cursor.close();
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        } finally {
+            indexHelper.close();
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        writer.fail();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/PermutingFrameTupleReference.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/PermutingFrameTupleReference.java
deleted file mode 100644
index 0b296f0..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/PermutingFrameTupleReference.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-public class PermutingFrameTupleReference implements IFrameTupleReference {
-	private IFrameTupleAccessor fta;
-	private int tIndex;
-	private int[] fieldPermutation;
-
-	public void setFieldPermutation(int[] fieldPermutation) {
-		this.fieldPermutation = fieldPermutation;
-	}
-
-	public void reset(IFrameTupleAccessor fta, int tIndex) {
-		this.fta = fta;
-		this.tIndex = tIndex;
-	}
-
-	@Override
-	public IFrameTupleAccessor getFrameTupleAccessor() {
-		return fta;
-	}
-
-	@Override
-	public int getTupleIndex() {
-		return tIndex;
-	}
-
-	@Override
-	public int getFieldCount() {
-		return fieldPermutation.length;
-	}
-
-	@Override
-	public byte[] getFieldData(int fIdx) {
-		return fta.getBuffer().array();
-	}
-
-	@Override
-	public int getFieldStart(int fIdx) {
-		return fta.getTupleStartOffset(tIndex) + fta.getFieldSlotsLength()
-				+ fta.getFieldStartOffset(tIndex, fieldPermutation[fIdx]);
-	}
-
-	@Override
-	public int getFieldLength(int fIdx) {
-		return fta.getFieldLength(tIndex, fieldPermutation[fIdx]);
-	}
-}
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 0020089..f7f57e6 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
@@ -22,8 +22,11 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class TreeIndexBulkLoadOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
@@ -31,22 +34,30 @@
 
     private final int[] fieldPermutation;
     private final float fillFactor;
+    private final boolean verifyInput;
+    private final long numElementsHint;
 
-    public TreeIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<IIndex> indexRegistryProvider, IFileSplitProvider fileSplitProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
-            float fillFactor, IIndexDataflowHelperFactory dataflowHelperFactory,
-            IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 1, 0, null, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
+    public TreeIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
+            float fillFactor, boolean verifyInput, long numElementsHint,
+            IIndexDataflowHelperFactory dataflowHelperFactory,
+            IModificationOperationCallbackFactory modificationOpCallbackFactory) {
+        super(spec, 1, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, false,
+                NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE,
+                modificationOpCallbackFactory);
         this.fieldPermutation = fieldPermutation;
         this.fillFactor = fillFactor;
+        this.verifyInput = verifyInput;
+        this.numElementsHint = numElementsHint;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new TreeIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation,
-                fillFactor, recordDescProvider);
+        return new IndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor, verifyInput,
+                numElementsHint, 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/dataflow/TreeIndexBulkLoadOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
deleted file mode 100644
index a2d78a4..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-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;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-
-public class TreeIndexBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
-    private float fillFactor;
-    private final TreeIndexDataflowHelper treeIndexHelper;
-    private FrameTupleAccessor accessor;
-    private IIndexBulkLoadContext bulkLoadCtx;
-    private ITreeIndex treeIndex;
-
-    private IRecordDescriptorProvider recordDescProvider;
-
-    private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
-
-    public TreeIndexBulkLoadOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
-            int partition, int[] fieldPermutation, float fillFactor, IRecordDescriptorProvider recordDescProvider) {
-        treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
-                opDesc, ctx, partition);
-        this.fillFactor = fillFactor;
-        this.recordDescProvider = recordDescProvider;
-        tuple.setFieldPermutation(fieldPermutation);
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
-                .getOperatorDescriptor();
-        RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), recDesc);
-        try {
-            treeIndexHelper.init(false);
-            treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
-            treeIndex.open(treeIndexHelper.getIndexFileId());
-            bulkLoadCtx = treeIndex.beginBulkLoad(fillFactor);
-        } catch (Exception e) {
-            // cleanup in case of failure
-            treeIndexHelper.deinit();
-            throw new HyracksDataException(e);
-        }
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        accessor.reset(buffer);
-        int tupleCount = accessor.getTupleCount();
-        for (int i = 0; i < tupleCount; i++) {
-            tuple.reset(accessor, i);
-            treeIndex.bulkLoadAddTuple(tuple, bulkLoadCtx);
-        }
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        try {
-            treeIndex.endBulkLoad(bulkLoadCtx);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        } finally {
-            treeIndexHelper.deinit();
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
index 075a6a4..8b7e81d 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2009-2010 by The Regents of the University of California
+ * Copyright 2009-2012 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
@@ -22,24 +22,30 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
 
 public class TreeIndexCreateOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     public TreeIndexCreateOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<IIndex> indexRegistryProvider, IFileSplitProvider fileSplitProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories,
-            IIndexDataflowHelperFactory dataflowHelperFactory, IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 0, 0, null, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
+            IIndexDataflowHelperFactory dataflowHelperFactory,
+            ILocalResourceFactoryProvider localResourceFactoryProvider,
+            IModificationOperationCallbackFactory modificationOpCallbackFactory) {
+        super(spec, 0, 0, null, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, false,
+                localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new TreeIndexCreateOperatorNodePushable(this, ctx, partition);
+        return new IndexCreateOperatorNodePushable(this, ctx, partition);
     }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorNodePushable.java
deleted file mode 100644
index 21348a0..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorNodePushable.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow;
-
-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.std.base.AbstractOperatorNodePushable;
-
-public class TreeIndexCreateOperatorNodePushable extends AbstractOperatorNodePushable {
-    protected final TreeIndexDataflowHelper treeIndexHelper;
-
-    public TreeIndexCreateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
-            int partition) {
-        treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
-                opDesc, ctx, partition);
-    }
-
-    @Override
-    public void deinitialize() throws HyracksDataException {
-    }
-
-    @Override
-    public int getInputArity() {
-        return 0;
-    }
-
-    @Override
-    public IFrameWriter getInputFrameWriter(int index) {
-        return null;
-    }
-
-    @Override
-    public void initialize() throws HyracksDataException {
-        try {
-        	treeIndexHelper.init(true);
-        } finally {
-        	treeIndexHelper.deinit();
-        }
-    }
-
-    @Override
-    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java
index 10d1077..711dfe0 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java
@@ -17,22 +17,17 @@
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 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.impls.TreeDiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
 
 public abstract class TreeIndexDataflowHelper extends IndexDataflowHelper {
-    protected ITreeIndexOperatorDescriptor treeOpDesc;
 
     public TreeIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition) {
         super(opDesc, ctx, partition);
-        this.treeOpDesc = (ITreeIndexOperatorDescriptor) opDesc;
     }
 
-    public abstract ITreeIndex createIndexInstance() throws HyracksDataException;
-
     public ITreeIndexCursor createDiskOrderScanCursor(ITreeIndexFrame leafFrame) throws HyracksDataException {
-        return new TreeDiskOrderScanCursor(leafFrame);
+        return new TreeIndexDiskOrderScanCursor(leafFrame);
     }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index 324485e..a8644e4 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -22,19 +22,23 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class TreeIndexDiskOrderScanOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     public TreeIndexDiskOrderScanOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
-            IIndexDataflowHelperFactory dataflowHelperFactory, IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits, null,
-                dataflowHelperFactory, null, false, opCallbackProvider);
+            IIndexDataflowHelperFactory dataflowHelperFactory, ISearchOperationCallbackFactory searchOpCallbackProvider) {
+        super(spec, 0, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits, null, null,
+                dataflowHelperFactory, null, false, NoOpLocalResourceFactoryProvider.INSTANCE,
+                searchOpCallbackProvider, NoOpOperationCallbackFactory.INSTANCE);
     }
 
     @Override
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index d02a570..a861af7 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -24,36 +24,44 @@
 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.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 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.impls.NoOpOperationCallback;
 
 public class TreeIndexDiskOrderScanOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+    private final AbstractTreeIndexOperatorDescriptor opDesc;
+    private final IHyracksTaskContext ctx;
     private final TreeIndexDataflowHelper treeIndexHelper;
     private ITreeIndex treeIndex;
 
     public TreeIndexDiskOrderScanOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition) {
-        treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
-                opDesc, ctx, partition);
+        this.opDesc = opDesc;
+        this.ctx = ctx;
+        this.treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory()
+                .createIndexDataflowHelper(opDesc, ctx, partition);
     }
 
     @Override
     public void initialize() throws HyracksDataException {
+        treeIndexHelper.open();
+        treeIndex = (ITreeIndex) treeIndexHelper.getIndexInstance();
         try {
-            treeIndexHelper.init(false);
-            treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
             ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame();
             ITreeIndexCursor cursor = treeIndexHelper.createDiskOrderScanCursor(cursorFrame);
-            ITreeIndexAccessor indexAccessor = (ITreeIndexAccessor) treeIndex.createAccessor();
+            ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory().createSearchOperationCallback(
+                    treeIndexHelper.getResourceID(), ctx);
+            ITreeIndexAccessor indexAccessor = (ITreeIndexAccessor) treeIndex.createAccessor(
+                    NoOpOperationCallback.INSTANCE, searchCallback);
             writer.open();
             try {
                 indexAccessor.diskOrderScan(cursor);
                 int fieldCount = treeIndex.getFieldCount();
-                ByteBuffer frame = treeIndexHelper.getHyracksTaskContext().allocateFrame();
-                FrameTupleAppender appender = new FrameTupleAppender(treeIndexHelper.getHyracksTaskContext()
-                        .getFrameSize());
+                ByteBuffer frame = ctx.allocateFrame();
+                FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
                 appender.reset(frame, true);
                 ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
                 DataOutput dos = tb.getDataOutput();
@@ -87,13 +95,13 @@
                 writer.close();
             }
         } catch (Exception e) {
-            deinitialize();
+            treeIndexHelper.close();
             throw new HyracksDataException(e);
         }
     }
 
     @Override
     public void deinitialize() throws HyracksDataException {
-        treeIndexHelper.deinit();
+        treeIndexHelper.close();
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
deleted file mode 100644
index 7c58031..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public class TreeIndexDropOperatorDescriptor extends
-		AbstractSingleActivityOperatorDescriptor {
-
-	private static final long serialVersionUID = 1L;
-
-	private IStorageManagerInterface storageManager;
-	private IIndexRegistryProvider<IIndex> treeIndexRegistryProvider;
-	private IFileSplitProvider fileSplitProvider;
-
-	public TreeIndexDropOperatorDescriptor(IOperatorDescriptorRegistry spec,
-			IStorageManagerInterface storageManager,
-			IIndexRegistryProvider<IIndex> treeIndexRegistryProvider,
-			IFileSplitProvider fileSplitProvider) {
-		super(spec, 0, 0);
-		this.storageManager = storageManager;
-		this.treeIndexRegistryProvider = treeIndexRegistryProvider;
-		this.fileSplitProvider = fileSplitProvider;
-	}
-
-	@Override
-	public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-			IRecordDescriptorProvider recordDescProvider,
-			int partition, int nPartitions) {
-		return new TreeIndexDropOperatorNodePushable(ctx, storageManager,
-				treeIndexRegistryProvider, fileSplitProvider, partition);
-	}
-}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
deleted file mode 100644
index 5f3c0b5..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-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.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-
-public class TreeIndexDropOperatorNodePushable extends AbstractOperatorNodePushable {
-    private static final Logger LOGGER = Logger.getLogger(TreeIndexDropOperatorNodePushable.class.getName());
-
-    private final IHyracksTaskContext ctx;
-    private IIndexRegistryProvider<IIndex> treeIndexRegistryProvider;
-    private IStorageManagerInterface storageManager;
-    private IFileSplitProvider fileSplitProvider;
-    private int partition;
-
-    public TreeIndexDropOperatorNodePushable(IHyracksTaskContext ctx, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<IIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider,
-            int partition) {
-        this.ctx = ctx;
-        this.storageManager = storageManager;
-        this.treeIndexRegistryProvider = treeIndexRegistryProvider;
-        this.fileSplitProvider = fileSplitProvider;
-        this.partition = partition;
-    }
-
-    @Override
-    public void deinitialize() throws HyracksDataException {
-    }
-
-    @Override
-    public int getInputArity() {
-        return 0;
-    }
-
-    @Override
-    public IFrameWriter getInputFrameWriter(int index) {
-        return null;
-    }
-
-    @Override
-    public void initialize() throws HyracksDataException {
-        try {
-            IndexRegistry<IIndex> treeIndexRegistry = treeIndexRegistryProvider.getRegistry(ctx);
-            IBufferCache bufferCache = storageManager.getBufferCache(ctx);
-            IFileMapProvider fileMapProvider = storageManager.getFileMapProvider(ctx);
-
-            FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
-            int indexFileId = -1;
-            synchronized (fileMapProvider) {
-                boolean fileIsMapped = fileMapProvider.isMapped(f);
-                if (!fileIsMapped) {
-                    throw new HyracksDataException("Cannot drop Tree with name " + f.toString()
-                            + ". No file mapping exists.");
-                }
-                indexFileId = fileMapProvider.lookupFileId(f);
-            }
-            // Unregister tree instance.
-            synchronized (treeIndexRegistry) {
-                treeIndexRegistry.unregister(indexFileId);
-            }
-
-            // remove name to id mapping
-            bufferCache.deleteFile(indexFileId, false);
-        }
-        // TODO: for the time being we don't throw,
-        // with proper exception handling (no hanging job problem) we should
-        // throw
-        catch (Exception e) {
-            if (LOGGER.isLoggable(Level.WARNING)) {
-                LOGGER.warning("Tree Drop Operator Failed Due To Exception: " + e.getMessage());
-            }
-        }
-    }
-
-    @Override
-    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index a615386..84e6090 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2009-2010 by The Regents of the University of California
+ * Copyright 2009-2012 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
@@ -23,26 +23,31 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+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.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class TreeIndexInsertUpdateDeleteOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     private final int[] fieldPermutation;
-
-    private IndexOp op;
+    private final IndexOperation op;
 
     public TreeIndexInsertUpdateDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation, IndexOp op,
-            IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory, IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, tupleFilterFactory, false, opCallbackProvider);
+            IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
+            IndexOperation op, IIndexDataflowHelperFactory dataflowHelperFactory,
+            ITupleFilterFactory tupleFilterFactory, IModificationOperationCallbackFactory modificationOpCallbackProvider) {
+        super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, tupleFilterFactory, false,
+                NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE,
+                modificationOpCallbackProvider);
         this.fieldPermutation = fieldPermutation;
         this.op = op;
     }
@@ -50,7 +55,7 @@
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new TreeIndexInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
+        return new IndexInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
                 recordDescProvider, op);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
deleted file mode 100644
index e05568f..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-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;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilter;
-import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-
-public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-    private final TreeIndexDataflowHelper treeIndexHelper;
-    private FrameTupleAccessor accessor;
-    private final IRecordDescriptorProvider recordDescProvider;
-    private final IndexOp op;
-    private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
-    private FrameTupleReference frameTuple;
-    private ByteBuffer writeBuffer;
-    private IIndexAccessor indexAccessor;
-    private ITupleFilter tupleFilter;
-
-    public TreeIndexInsertUpdateDeleteOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
-            IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
-            IRecordDescriptorProvider recordDescProvider, IndexOp op) {
-        treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
-                opDesc, ctx, partition);
-        this.recordDescProvider = recordDescProvider;
-        this.op = op;
-        tuple.setFieldPermutation(fieldPermutation);
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
-                .getOperatorDescriptor();
-        RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
-        writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
-        writer.open();
-        try {
-            treeIndexHelper.init(false);
-            ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
-            indexAccessor = treeIndex.createAccessor();
-            ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
-            if (tupleFilterFactory != null) {
-                tupleFilter = tupleFilterFactory.createTupleFilter(treeIndexHelper.ctx);
-                frameTuple = new FrameTupleReference();
-            }
-        } catch (Exception e) {
-            // cleanup in case of failure
-            treeIndexHelper.deinit();
-            throw new HyracksDataException(e);
-        }
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        accessor.reset(buffer);
-        int tupleCount = accessor.getTupleCount();
-        for (int i = 0; i < tupleCount; i++) {
-            try {
-                if (tupleFilter != null) {
-                    frameTuple.reset(accessor, i);
-                    if (!tupleFilter.accept(frameTuple)) {
-                        continue;
-                    }
-                }
-                tuple.reset(accessor, i);
-                switch (op) {
-                    case INSERT: {
-                        indexAccessor.insert(tuple);
-                        break;
-                    }
-                    case UPDATE: {
-                        indexAccessor.update(tuple);
-                        break;
-                    }
-                    case UPSERT: {
-                        indexAccessor.upsert(tuple);
-                        break;
-                    }
-                    case DELETE: {
-                        indexAccessor.delete(tuple);
-                        break;
-                    }
-                    default: {
-                        throw new HyracksDataException("Unsupported operation " + op
-                                + " in tree index InsertUpdateDelete operator");
-                    }
-                }
-            } catch (HyracksDataException e) {
-                throw e;
-            } catch (Exception e) {
-                throw new HyracksDataException(e);
-            }
-        }
-        // Pass a copy of the frame to next op.
-        System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
-        FrameUtils.flushFrame(writeBuffer, writer);
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        try {
-            writer.close();
-        } finally {
-            treeIndexHelper.deinit();
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        writer.fail();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexSearchOperatorNodePushable.java
deleted file mode 100644
index 5c19483..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexSearchOperatorNodePushable.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow;
-
-import java.io.DataOutput;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-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;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-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.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-
-public abstract class TreeIndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-    protected final TreeIndexDataflowHelper treeIndexHelper;
-    protected FrameTupleAccessor accessor;
-
-    protected ByteBuffer writeBuffer;
-    protected FrameTupleAppender appender;
-    protected ArrayTupleBuilder tb;
-    protected DataOutput dos;
-
-    protected ITreeIndex treeIndex;
-    protected ISearchPredicate searchPred;
-    protected IIndexCursor cursor;
-    protected ITreeIndexFrame cursorFrame;
-    protected IIndexAccessor indexAccessor;
-
-    protected final RecordDescriptor inputRecDesc;
-    protected final boolean retainInput;
-    protected FrameTupleReference frameTuple;
-
-    public TreeIndexSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
-            int partition, IRecordDescriptorProvider recordDescProvider) {
-        treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
-                opDesc, ctx, partition);
-        this.retainInput = treeIndexHelper.getOperatorDescriptor().getRetainInput();
-        inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-    }
-
-    protected abstract ISearchPredicate createSearchPredicate();
-
-    protected abstract void resetSearchPredicate(int tupleIndex);
-
-    protected IIndexCursor createCursor() {
-        return indexAccessor.createSearchCursor();
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
-        writer.open();
-        try {        	
-            treeIndexHelper.init(false);
-            treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
-            cursorFrame = treeIndex.getLeafFrameFactory().createFrame();
-            searchPred = createSearchPredicate();
-            writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
-            tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
-            dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(treeIndexHelper.getHyracksTaskContext().getFrameSize());
-            appender.reset(writeBuffer, true);
-            indexAccessor = treeIndex.createAccessor();
-            cursor = createCursor();
-            if (retainInput) {
-            	frameTuple = new FrameTupleReference();
-            }
-        } catch (Exception e) {
-            treeIndexHelper.deinit();
-            throw new HyracksDataException(e);
-        }
-    }
-
-    protected void writeSearchResults(int tupleIndex) throws Exception {
-        while (cursor.hasNext()) {
-            tb.reset();
-            cursor.next();
-            if (retainInput) {
-            	frameTuple.reset(accessor, tupleIndex);
-                for (int i = 0; i < frameTuple.getFieldCount(); i++) {
-                	dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
-                    tb.addFieldEndOffset();
-                }
-            }
-            ITupleReference tuple = cursor.getTuple();
-            for (int i = 0; i < tuple.getFieldCount(); i++) {
-                dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
-                tb.addFieldEndOffset();
-            }
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                FrameUtils.flushFrame(writeBuffer, writer);
-                appender.reset(writeBuffer, true);
-                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    throw new IllegalStateException();
-                }
-            }
-        }
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        accessor.reset(buffer);
-        int tupleCount = accessor.getTupleCount();
-        try {
-            for (int i = 0; i < tupleCount; i++) {
-                resetSearchPredicate(i);
-                cursor.reset();
-                indexAccessor.search(cursor, searchPred);
-                writeSearchResults(i);
-            }
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        try {
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(writeBuffer, writer);
-            }
-            writer.close();
-            try {
-                cursor.close();
-            } catch (Exception e) {
-                throw new HyracksDataException(e);
-            }
-        } finally {
-            treeIndexHelper.deinit();
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        writer.fail();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
index 6bf0983..b589f96 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -24,8 +24,11 @@
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class TreeIndexStatsOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
@@ -34,11 +37,13 @@
             new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
 
     public TreeIndexStatsOperatorDescriptor(IOperatorDescriptorRegistry spec, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<IIndex> indexRegistryProvider, IFileSplitProvider fileSplitProvider,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories,
-            IIndexDataflowHelperFactory dataflowHelperFactory, IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, IFileSplitProvider fileSplitProvider,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields,
+            IIndexDataflowHelperFactory dataflowHelperFactory, ISearchOperationCallbackFactory searchOpCallbackProvider) {
+        super(spec, 0, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, false,
+                NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider,
+                NoOpOperationCallbackFactory.INSTANCE);
     }
 
     @Override
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
index 50486f2..09d357d 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
@@ -29,17 +29,21 @@
 import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStats;
 import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStatsGatherer;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class TreeIndexStatsOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
-    private final TreeIndexDataflowHelper treeIndexHelper;
+    private final AbstractTreeIndexOperatorDescriptor opDesc;
     private final IHyracksTaskContext ctx;
+    private final TreeIndexDataflowHelper treeIndexHelper;
     private TreeIndexStatsGatherer statsGatherer;
 
     public TreeIndexStatsOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition) {
-        treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
-                opDesc, ctx, partition);
+        this.opDesc = opDesc;
         this.ctx = ctx;
+        this.treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory()
+                .createIndexDataflowHelper(opDesc, ctx, partition);
+
     }
 
     @Override
@@ -53,13 +57,15 @@
 
     @Override
     public void initialize() throws HyracksDataException {
+        treeIndexHelper.open();
+        ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndexInstance();
         try {
             writer.open();
-            treeIndexHelper.init(false);
-            ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
-            IBufferCache bufferCache = treeIndexHelper.getOperatorDescriptor().getStorageManager().getBufferCache(ctx);
-            statsGatherer = new TreeIndexStatsGatherer(bufferCache, treeIndex.getFreePageManager(),
-                    treeIndexHelper.getIndexFileId(), treeIndex.getRootPageId());
+            IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
+            IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
+            int indexFileId = fileMapProvider.lookupFileId(treeIndexHelper.getFileReference());
+            statsGatherer = new TreeIndexStatsGatherer(bufferCache, treeIndex.getFreePageManager(), indexFileId,
+                    treeIndex.getRootPageId());
             TreeIndexStats stats = statsGatherer.gatherStats(treeIndex.getLeafFrameFactory().createFrame(), treeIndex
                     .getInteriorFrameFactory().createFrame(), treeIndex.getFreePageManager().getMetaDataFrameFactory()
                     .createFrame());
@@ -77,13 +83,10 @@
             }
             FrameUtils.flushFrame(frame, writer);
         } catch (Exception e) {
-            try {
-                treeIndexHelper.deinit();
-            } finally {
-                writer.fail();
-            }
+            writer.fail();
         } finally {
             writer.close();
+            treeIndexHelper.close();
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java
index e8d3d56..6fff620 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 import java.io.IOException;
@@ -23,7 +38,8 @@
     private TupleBatch[] tupleBatches;
     private int ringPos;
     
-    public DataGenThread(int numConsumers, int maxNumBatches, int batchSize, ISerializerDeserializer[] fieldSerdes, int payloadSize, int rndSeed, int maxOutstandingBatches, boolean sorted) {
+    public DataGenThread(int numConsumers, int maxNumBatches, int batchSize, ISerializerDeserializer[] fieldSerdes,
+            int payloadSize, int rndSeed, int maxOutstandingBatches, boolean sorted) {
         this.maxNumBatches = maxNumBatches;
         this.maxOutstandingBatches = maxOutstandingBatches;
         rnd = new Random(rndSeed);
@@ -36,6 +52,19 @@
         ringPos = 0;
     }
     
+    public DataGenThread(int numConsumers, int maxNumBatches, int batchSize, ISerializerDeserializer[] fieldSerdes,
+            IFieldValueGenerator[] fieldGens, int rndSeed, int maxOutstandingBatches) {
+        this.maxNumBatches = maxNumBatches;
+        this.maxOutstandingBatches = maxOutstandingBatches;
+        rnd = new Random(rndSeed);
+        tupleBatches = new TupleBatch[maxOutstandingBatches];
+        for (int i = 0; i < maxOutstandingBatches; i++) {
+            tupleBatches[i] = new TupleBatch(batchSize, fieldGens, fieldSerdes, 0);
+        }
+        tupleBatchQueue = new LinkedBlockingQueue<TupleBatch>(maxOutstandingBatches);
+        ringPos = 0;
+    }
+    
     @Override
     public void run() {
         while(numBatches < maxNumBatches) {
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenUtils.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenUtils.java
index fdbaa3e..b3914e6 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenUtils.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenUtils.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 import java.util.Random;
@@ -32,7 +47,6 @@
         } else if (serde instanceof UTF8StringSerializerDeserializer) {
             return new StringFieldValueGenerator(20, rnd);
         }
-        System.out.println("NULL");
         return null;
     }
     
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DocumentStringFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DocumentStringFieldValueGenerator.java
new file mode 100644
index 0000000..b12bb7d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DocumentStringFieldValueGenerator.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+public class DocumentStringFieldValueGenerator implements IFieldValueGenerator<String> {
+    private final String FIRST_NAMES_FILE = "dist.all.first.cleaned";
+    private final String LAST_NAMES_FILE = "dist.all.last.cleaned";
+
+    private final int docMinWords;
+    private final int docMaxWords;
+    private final int maxDictionarySize;
+    private final Random rnd;
+    private int[] cumulIntRanges;
+
+    private List<String> tokenDict = new ArrayList<String>();
+
+    public DocumentStringFieldValueGenerator(int docMinWords, int docMaxWords, int maxDictionarySize, Random rnd)
+            throws IOException {
+        this.docMinWords = docMinWords;
+        this.docMaxWords = docMaxWords;
+        this.maxDictionarySize = maxDictionarySize;
+        this.rnd = rnd;
+        initDictionary();
+        double[] zipfProbDist = ProbabilityHelper.getZipfProbDist(tokenDict.size(), 1);
+        cumulIntRanges = ProbabilityHelper.getCumulIntRanges(zipfProbDist);
+    }
+
+    private void initDictionary() throws IOException {
+        String line;
+        int count = 0;
+
+        // Read first names from data file.
+        InputStream firstNamesIn = this.getClass().getClassLoader().getResourceAsStream(FIRST_NAMES_FILE);
+        BufferedReader firstNamesReader = new BufferedReader(new InputStreamReader(firstNamesIn));
+        try {
+            while (count < maxDictionarySize && (line = firstNamesReader.readLine()) != null) {
+                tokenDict.add(line.trim());
+                count++;
+            }
+        } finally {
+            firstNamesReader.close();
+        }
+
+        // Read last names from data file.
+        InputStream lastNamesIn = this.getClass().getClassLoader().getResourceAsStream(LAST_NAMES_FILE);
+        BufferedReader lastNamesReader = new BufferedReader(new InputStreamReader(lastNamesIn));
+        try {
+            while (count < maxDictionarySize && (line = lastNamesReader.readLine()) != null) {
+                tokenDict.add(line.trim());
+                count++;
+            }
+        } finally {
+            lastNamesReader.close();
+        }
+    }
+
+    @Override
+    public String next() {
+        StringBuilder strBuilder = new StringBuilder();
+        int numWords = Math.abs(rnd.nextInt()) % (docMaxWords - docMinWords + 1) + docMinWords;
+        for (int i = 0; i < numWords; i++) {
+            int ix = ProbabilityHelper.choose(cumulIntRanges, rnd.nextInt());
+            strBuilder.append(tokenDict.get(ix));
+            if (i != numWords - 1) {
+                strBuilder.append(" ");
+            }
+        }
+        return strBuilder.toString();
+    }
+
+    public List<String> getTokenDictionary() {
+        return tokenDict;
+    }
+
+    @Override
+    public void reset() {
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DoubleFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DoubleFieldValueGenerator.java
index fcac93a..c98c249 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DoubleFieldValueGenerator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DoubleFieldValueGenerator.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 import java.util.Random;
@@ -13,4 +28,8 @@
     public Double next() {
         return rnd.nextDouble();
     }
+
+    @Override
+    public void reset() {
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/FloatFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/FloatFieldValueGenerator.java
index 6f21c77..7c3ff81 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/FloatFieldValueGenerator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/FloatFieldValueGenerator.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 import java.util.Random;
@@ -13,4 +28,8 @@
     public Float next() {
         return rnd.nextFloat();
     }
+
+    @Override
+    public void reset() {
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/IFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/IFieldValueGenerator.java
index ee0d30b..dfeead6 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/IFieldValueGenerator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/IFieldValueGenerator.java
@@ -1,5 +1,21 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 public interface IFieldValueGenerator<T> {
     public T next();
+    public void reset();
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/IntegerFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/IntegerFieldValueGenerator.java
index 134b1f7..cd6e2a6 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/IntegerFieldValueGenerator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/IntegerFieldValueGenerator.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 import java.util.Random;
@@ -13,4 +28,8 @@
     public Integer next() {
         return rnd.nextInt();
     }
+
+    @Override
+    public void reset() {
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/PersonNameFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/PersonNameFieldValueGenerator.java
new file mode 100644
index 0000000..6b86278
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/PersonNameFieldValueGenerator.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+public class PersonNameFieldValueGenerator implements IFieldValueGenerator<String> {
+    private final String FIRST_NAMES_FILE = "dist.all.first.cleaned";
+    private final String LAST_NAMES_FILE = "dist.all.last.cleaned";
+
+    private final Random rnd;
+    private final double middleInitialProb;
+    private final String letters = "ABCDEFGHIJKLMNOPQRSTUVWXYZ";
+
+    private List<String> firstNames = new ArrayList<String>();
+    private List<String> lastNames = new ArrayList<String>();
+
+    public PersonNameFieldValueGenerator(Random rnd, double middleInitialProb)
+            throws IOException {
+        this.rnd = rnd;
+        this.middleInitialProb = middleInitialProb;
+        initNames();
+    }
+
+    private void initNames() throws IOException {
+        String line;
+
+        // Read first names from data file.
+        InputStream firstNamesIn = this.getClass().getClassLoader().getResourceAsStream(FIRST_NAMES_FILE);
+        BufferedReader firstNamesReader = new BufferedReader(new InputStreamReader(firstNamesIn));
+        try {
+            while ((line = firstNamesReader.readLine()) != null) {
+                firstNames.add(line.trim());
+            }
+        } finally {
+            firstNamesReader.close();
+        }
+
+        // Read last names from data file.
+        InputStream lastNamesIn = this.getClass().getClassLoader().getResourceAsStream(LAST_NAMES_FILE);
+        BufferedReader lastNamesReader = new BufferedReader(new InputStreamReader(lastNamesIn));
+        try {
+            while ((line = lastNamesReader.readLine()) != null) {
+                lastNames.add(line.trim());
+            }
+        } finally {
+            lastNamesReader.close();
+        }
+    }
+
+    @Override
+    public String next() {
+        StringBuilder strBuilder = new StringBuilder();
+
+        // First name.
+        int fix = Math.abs(rnd.nextInt()) % firstNames.size();
+        strBuilder.append(firstNames.get(fix));
+        strBuilder.append(" ");
+        
+        // Optional middle initial.
+        double d = Math.abs(rnd.nextDouble());
+        if (d <= middleInitialProb) {
+            int mix = Math.abs(rnd.nextInt()) % letters.length();
+            strBuilder.append(letters.charAt(mix));
+            strBuilder.append(". ");
+        }
+        
+        // Last name.
+        int lix = Math.abs(rnd.nextInt()) % lastNames.size();
+        strBuilder.append(lastNames.get(lix));
+        
+        return strBuilder.toString();
+    }
+
+    @Override
+    public void reset() {
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/ProbabilityHelper.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/ProbabilityHelper.java
new file mode 100644
index 0000000..1674681
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/ProbabilityHelper.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
+
+import java.util.Arrays;
+
+public class ProbabilityHelper {
+    public static double[] getUniformProbDist(int numChoices) {
+        double[] probDist = new double[numChoices];
+        for (int i = 0; i < numChoices; i++) {
+            probDist[i] = 1.0 / (double) numChoices;
+        }
+        return probDist;
+    }
+
+    public static double[] getZipfProbDist(int numChoices, int zipfSkew) {
+        double[] probDist = new double[numChoices];
+        double divisor = 0;
+        for (int i = 1; i <= numChoices; i++) {
+            divisor += 1.0 / (double) Math.pow((double) i, (double) zipfSkew);
+        }
+        for (int i = 1; i <= numChoices; i++) {
+            probDist[i - 1] = (1.0 / (double) Math.pow((double) i, (double) zipfSkew)) / divisor;
+        }
+        return probDist;
+    }
+
+    public static int[] getCumulIntRanges(double[] probDist) {
+        int[] opRanges = new int[probDist.length];
+        if (opRanges.length > 1) {
+            opRanges[0] = (int) Math.floor(Integer.MAX_VALUE * probDist[0]);
+            for (int i = 1; i < opRanges.length - 1; i++) {
+                opRanges[i] = opRanges[i - 1] + (int) Math.floor(Integer.MAX_VALUE * probDist[i]);
+            }
+            opRanges[opRanges.length - 1] = Integer.MAX_VALUE;
+        } else {
+            opRanges[0] = Integer.MAX_VALUE;
+        }
+        return opRanges;
+    }
+
+    public static int choose(int[] cumulIntRanges, int randomInt) {
+        int rndVal = Math.abs(randomInt);
+        int ix = Arrays.binarySearch(cumulIntRanges, rndVal);
+        if (ix < 0) {
+            ix = -ix - 1;
+        }
+        return ix;
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedDoubleFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedDoubleFieldValueGenerator.java
index 4193811..e93b8de 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedDoubleFieldValueGenerator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedDoubleFieldValueGenerator.java
@@ -1,17 +1,41 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 public class SortedDoubleFieldValueGenerator implements IFieldValueGenerator<Double> {
-    private double val = 0.0d;
-
+    private double val;
+    private final double startVal;
+    
     public SortedDoubleFieldValueGenerator() {
+        startVal = 0.0d;
+        reset();
     }
     
     public SortedDoubleFieldValueGenerator(double startVal) {
-        val = startVal;
+        this.startVal = startVal;
+        reset();
     }
     
     @Override
     public Double next() {
         return val++;
     }
+
+    @Override
+    public void reset() {
+        val = startVal;        
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedFloatFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedFloatFieldValueGenerator.java
index 1f6b315..fb163e1 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedFloatFieldValueGenerator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedFloatFieldValueGenerator.java
@@ -1,17 +1,41 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 public class SortedFloatFieldValueGenerator implements IFieldValueGenerator<Float> {
     private float val = 0.0f;
-
+    private final float startVal;
+    
     public SortedFloatFieldValueGenerator() {
+        startVal = 0.0f;
+        reset();
     }
     
     public SortedFloatFieldValueGenerator(float startVal) {
-        val = startVal;
+        this.startVal = startVal;
+        reset();
     }
     
     @Override
     public Float next() {
         return val++;
     }
+
+    @Override
+    public void reset() {
+        val = startVal;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedIntegerFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedIntegerFieldValueGenerator.java
index 8f7fdcf..a036772 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedIntegerFieldValueGenerator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/SortedIntegerFieldValueGenerator.java
@@ -1,17 +1,41 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 public class SortedIntegerFieldValueGenerator implements IFieldValueGenerator<Integer> {
     private int val = 0;
+    private final int startVal;
 
     public SortedIntegerFieldValueGenerator() {
+        startVal = 0;
+        reset();
     }
     
     public SortedIntegerFieldValueGenerator(int startVal) {
-        val = startVal;
+        this.startVal = startVal;
+        reset();
     }
     
     @Override
     public Integer next() {
         return val++;
     }
+
+    @Override
+    public void reset() {
+        val = startVal;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/StringFieldValueGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/StringFieldValueGenerator.java
index 0218542..6bf01a4 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/StringFieldValueGenerator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/StringFieldValueGenerator.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 import java.util.Random;
@@ -24,4 +39,8 @@
         }
         return strBuilder.toString();
     }
+
+    @Override
+    public void reset() {
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/TupleBatch.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/TupleBatch.java
index bfa523f..375aeb0 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/TupleBatch.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/TupleBatch.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 import java.io.IOException;
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/TupleGenerator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/TupleGenerator.java
index 2801205..d0a1062 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/TupleGenerator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/TupleGenerator.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2012 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.common.datagen;
 
 import java.io.DataOutput;
@@ -48,4 +63,18 @@
     public ITupleReference get() {
         return tuple;
     }
+    
+    public void reset() {
+        for (IFieldValueGenerator fieldGen : fieldGens) {
+            fieldGen.reset();
+        }
+    }
+    
+    public ISerializerDeserializer[] getFieldSerdes() {
+        return fieldSerdes;
+    }
+    
+    public IFieldValueGenerator[] getFieldGens() {
+        return fieldGens;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrame.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrame.java
index 31c674d..1b8bc15 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrame.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrame.java
@@ -36,6 +36,7 @@
 	protected static final int levelOff = maxPageOff + 12; //20
 	protected static final int nextPageOff = levelOff + 1; // 21
 	protected static final int validOff = nextPageOff + 4; // 25
+	protected static final int lsnOff = validOff + 4; // 29
 
 	protected ICachedPage page = null;
 	protected ByteBuffer buf = null;
@@ -101,7 +102,7 @@
 	@Override
 	public void initBuffer(byte level) {
 		buf.putInt(tupleCountOff, 0);
-		buf.putInt(freeSpaceOff, validOff + 4);
+		buf.putInt(freeSpaceOff, lsnOff + 4);
 		//buf.putInt(maxPageOff, -1);
 		buf.put(levelOff, level);
 		buf.putInt(nextPageOff, -1);
@@ -131,4 +132,14 @@
             buf.putInt(validOff, 0);
         }
     }
+
+    @Override
+    public long getLSN() {
+        return buf.getLong(lsnOff);
+    }
+
+    @Override
+    public void setLSN(long lsn) {
+        buf.putLong(lsnOff, lsn);
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
index e2e28fd..31ce573 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
@@ -111,7 +111,7 @@
         resetSpaceParams();
         int tupleCount = buf.getInt(tupleCountOff);
         int freeSpace = buf.getInt(freeSpaceOff);
-		// Sort the slots by the tuple offset they point to.
+        // Sort the slots by the tuple offset they point to.
         ArrayList<SlotOffTupleOff> sortedTupleOffs = new ArrayList<SlotOffTupleOff>();
         sortedTupleOffs.ensureCapacity(tupleCount);
         for (int i = 0; i < tupleCount; i++) {
@@ -121,7 +121,7 @@
         }
         Collections.sort(sortedTupleOffs);
         // Iterate over the sorted slots, and move their corresponding tuples to
-     	// the left, reclaiming free space.
+        // the left, reclaiming free space.
         for (int i = 0; i < sortedTupleOffs.size(); i++) {
             int tupleOff = sortedTupleOffs.get(i).tupleOff;
             frameTuple.resetByTupleOffset(buf, tupleOff);
@@ -132,7 +132,7 @@
             slotManager.setSlot(sortedTupleOffs.get(i).slotOff, freeSpace);
             freeSpace += tupleLength;
         }
-		// Update contiguous free space pointer and total free space indicator.
+        // Update contiguous free space pointer and total free space indicator.
         buf.putInt(freeSpaceOff, freeSpace);
         buf.putInt(totalFreeSpaceOff, buf.capacity() - freeSpace - tupleCount * slotManager.getSlotSize());
         return false;
@@ -159,7 +159,7 @@
     public FrameOpSpaceStatus hasSpaceInsert(ITupleReference tuple) {
         int bytesRequired = tupleWriter.bytesRequired(tuple);
         // Enough space in the contiguous space region?
-        if (bytesRequired + slotManager.getSlotSize() <= buf.capacity() - buf.getInt(freeSpaceOff) 
+        if (bytesRequired + slotManager.getSlotSize() <= buf.capacity() - buf.getInt(freeSpaceOff)
                 - (buf.getInt(tupleCountOff) * slotManager.getSlotSize())) {
             return FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE;
         }
@@ -172,23 +172,24 @@
 
     @Override
     public FrameOpSpaceStatus hasSpaceUpdate(ITupleReference newTuple, int oldTupleIndex) {
-    	frameTuple.resetByTupleIndex(this, oldTupleIndex);
-    	int oldTupleBytes = frameTuple.getTupleSize();
-    	int newTupleBytes = tupleWriter.bytesRequired(newTuple);
-    	int additionalBytesRequired = newTupleBytes - oldTupleBytes;
-    	// Enough space for an in-place update?
-    	if (additionalBytesRequired <= 0) {
-    		return FrameOpSpaceStatus.SUFFICIENT_INPLACE_SPACE;
-    	}
-    	// Enough space if we delete the old tuple and insert the new one without compaction? 
-    	if (newTupleBytes <= buf.capacity() - buf.getInt(freeSpaceOff)
+        frameTuple.resetByTupleIndex(this, oldTupleIndex);
+        int oldTupleBytes = frameTuple.getTupleSize();
+        int newTupleBytes = tupleWriter.bytesRequired(newTuple);
+        int additionalBytesRequired = newTupleBytes - oldTupleBytes;
+        // Enough space for an in-place update?
+        if (additionalBytesRequired <= 0) {
+            return FrameOpSpaceStatus.SUFFICIENT_INPLACE_SPACE;
+        }
+        // Enough space if we delete the old tuple and insert the new one
+        // without compaction?
+        if (newTupleBytes <= buf.capacity() - buf.getInt(freeSpaceOff)
                 - (buf.getInt(tupleCountOff) * slotManager.getSlotSize())) {
-    		return FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE;
-    	}
-    	// Enough space if we delete the old tuple and compact?
-    	if (additionalBytesRequired <= buf.getInt(totalFreeSpaceOff)) {
-    		return FrameOpSpaceStatus.SUFFICIENT_SPACE;
-    	}
+            return FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE;
+        }
+        // Enough space if we delete the old tuple and compact?
+        if (additionalBytesRequired <= buf.getInt(totalFreeSpaceOff)) {
+            return FrameOpSpaceStatus.SUFFICIENT_SPACE;
+        }
         return FrameOpSpaceStatus.INSUFFICIENT_SPACE;
     }
 
@@ -208,35 +209,36 @@
 
     @Override
     public void update(ITupleReference newTuple, int oldTupleIndex, boolean inPlace) {
-    	frameTuple.resetByTupleIndex(this, oldTupleIndex);
-		int oldTupleBytes = frameTuple.getTupleSize();
-		int slotOff = slotManager.getSlotOff(oldTupleIndex);
-		int bytesWritten = 0;
-    	if (inPlace) {    		
-    		// Overwrite the old tuple in place.
-    		bytesWritten = tupleWriter.writeTuple(newTuple, buf.array(), buf.getInt(slotOff));
-    	} else {
-    		// Insert the new tuple at the end of the free space, and change the slot value (effectively "deleting" the old tuple).
-    		int newTupleOff = buf.getInt(freeSpaceOff);
-    		bytesWritten = tupleWriter.writeTuple(newTuple, buf.array(), newTupleOff);
-    		// Update slot value.
-    		buf.putInt(slotOff, newTupleOff);
-    		// Update contiguous free space pointer.
-    		buf.putInt(freeSpaceOff, newTupleOff + bytesWritten);
-    	}
-    	buf.putInt(totalFreeSpaceOff, buf.getInt(totalFreeSpaceOff) + oldTupleBytes - bytesWritten);
+        frameTuple.resetByTupleIndex(this, oldTupleIndex);
+        int oldTupleBytes = frameTuple.getTupleSize();
+        int slotOff = slotManager.getSlotOff(oldTupleIndex);
+        int bytesWritten = 0;
+        if (inPlace) {
+            // Overwrite the old tuple in place.
+            bytesWritten = tupleWriter.writeTuple(newTuple, buf.array(), buf.getInt(slotOff));
+        } else {
+            // Insert the new tuple at the end of the free space, and change the
+            // slot value (effectively "deleting" the old tuple).
+            int newTupleOff = buf.getInt(freeSpaceOff);
+            bytesWritten = tupleWriter.writeTuple(newTuple, buf.array(), newTupleOff);
+            // Update slot value.
+            buf.putInt(slotOff, newTupleOff);
+            // Update contiguous free space pointer.
+            buf.putInt(freeSpaceOff, newTupleOff + bytesWritten);
+        }
+        buf.putInt(totalFreeSpaceOff, buf.getInt(totalFreeSpaceOff) + oldTupleBytes - bytesWritten);
     }
 
     @Override
     public String printHeader() {
-    	StringBuilder strBuilder = new StringBuilder();
-    	strBuilder.append("pageLsnOff:        " + pageLsnOff + "\n");
-    	strBuilder.append("tupleCountOff:     " + tupleCountOff + "\n");
-    	strBuilder.append("freeSpaceOff:      " + freeSpaceOff + "\n");
-    	strBuilder.append("totalFreeSpaceOff: " + totalFreeSpaceOff + "\n");
-    	strBuilder.append("levelOff:          " + levelOff + "\n");
-    	strBuilder.append("smFlagOff:         " + smFlagOff + "\n");
-    	return strBuilder.toString();
+        StringBuilder strBuilder = new StringBuilder();
+        strBuilder.append("pageLsnOff:        " + pageLsnOff + "\n");
+        strBuilder.append("tupleCountOff:     " + tupleCountOff + "\n");
+        strBuilder.append("freeSpaceOff:      " + freeSpaceOff + "\n");
+        strBuilder.append("totalFreeSpaceOff: " + totalFreeSpaceOff + "\n");
+        strBuilder.append("levelOff:          " + levelOff + "\n");
+        strBuilder.append("smFlagOff:         " + smFlagOff + "\n");
+        return strBuilder.toString();
     }
 
     @Override
@@ -282,14 +284,13 @@
     public ITreeIndexTupleWriter getTupleWriter() {
         return tupleWriter;
     }
-    
+
     @Override
     public ITreeIndexTupleReference createTupleReference() {
-    	return tupleWriter.createTupleReference();
+        return tupleWriter.createTupleReference();
     }
-    
-	public int getFreeContiguousSpace() {
-		return buf.capacity() - getFreeSpaceOff()
-				- (getTupleCount() * slotManager.getSlotSize());
-	}
+
+    public int getFreeContiguousSpace() {
+        return buf.capacity() - getFreeSpaceOff() - (getTupleCount() * slotManager.getSlotSize());
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManagerFactory.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManagerFactory.java
index 157b563..1c373d5 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManagerFactory.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManagerFactory.java
@@ -16,19 +16,21 @@
 package edu.uci.ics.hyracks.storage.am.common.freepage;
 
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManagerFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 
-public class LinkedListFreePageManagerFactory {
+public class LinkedListFreePageManagerFactory implements IFreePageManagerFactory {
 
-	private final ITreeIndexMetaDataFrameFactory metaDataFrameFactory;
-	private final IBufferCache bufferCache;
-	
-	public LinkedListFreePageManagerFactory(IBufferCache bufferCache, ITreeIndexMetaDataFrameFactory metaDataFrameFactory) {
-		this.metaDataFrameFactory = metaDataFrameFactory;
-		this.bufferCache = bufferCache;
-	}
-	
+    private final ITreeIndexMetaDataFrameFactory metaDataFrameFactory;
+    private final IBufferCache bufferCache;
+
+    public LinkedListFreePageManagerFactory(IBufferCache bufferCache,
+            ITreeIndexMetaDataFrameFactory metaDataFrameFactory) {
+        this.metaDataFrameFactory = metaDataFrameFactory;
+        this.bufferCache = bufferCache;
+    }
+
     public IFreePageManager createFreePageManager() {
         return new LinkedListFreePageManager(bufferCache, 0, metaDataFrameFactory);
     }
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
new file mode 100644
index 0000000..9bf4a4f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -0,0 +1,359 @@
+/*

+ * Copyright 2009-2012 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.common.impls;

+

+import java.util.ArrayList;

+

+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;

+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;

+import edu.uci.ics.hyracks.api.io.FileReference;

+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;

+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;

+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;

+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;

+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;

+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;

+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;

+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;

+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;

+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.ophelpers.MultiComparator;

+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;

+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;

+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;

+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;

+

+public abstract class AbstractTreeIndex implements ITreeIndex {

+

+    protected final static int rootPage = 1;

+

+    protected final IBufferCache bufferCache;

+    protected final IFileMapProvider fileMapProvider;

+    protected final IFreePageManager freePageManager;

+

+    protected final ITreeIndexFrameFactory interiorFrameFactory;

+    protected final ITreeIndexFrameFactory leafFrameFactory;

+

+    protected final IBinaryComparatorFactory[] cmpFactories;

+    protected final int fieldCount;

+

+    protected FileReference file;

+    protected int fileId = -1;

+

+    private boolean isActivated = false;

+

+    public AbstractTreeIndex(IBufferCache bufferCache, IFileMapProvider fileMapProvider,

+            IFreePageManager freePageManager, ITreeIndexFrameFactory interiorFrameFactory,

+            ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount,

+            FileReference file) {

+        this.bufferCache = bufferCache;

+        this.fileMapProvider = fileMapProvider;

+        this.freePageManager = freePageManager;

+        this.interiorFrameFactory = interiorFrameFactory;

+        this.leafFrameFactory = leafFrameFactory;

+        this.cmpFactories = cmpFactories;

+        this.fieldCount = fieldCount;

+        this.file = file;

+    }

+

+    public synchronized void create() throws HyracksDataException {

+        if (isActivated) {

+            throw new HyracksDataException("Failed to create the index since it is activated.");

+        }

+

+        boolean fileIsMapped = false;

+        synchronized (fileMapProvider) {

+            fileIsMapped = fileMapProvider.isMapped(file);

+            if (!fileIsMapped) {

+                bufferCache.createFile(file);

+            }

+            fileId = fileMapProvider.lookupFileId(file);

+            try {

+                // Also creates the file if it doesn't exist yet.

+                bufferCache.openFile(fileId);

+            } catch (HyracksDataException e) {

+                // Revert state of buffer cache since file failed to open.

+                if (!fileIsMapped) {

+                    bufferCache.deleteFile(fileId, false);

+                }

+                throw e;

+            }

+        }

+

+        freePageManager.open(fileId);

+        initEmptyTree();

+        freePageManager.close();

+        bufferCache.closeFile(fileId);

+    }

+

+    private void initEmptyTree() throws HyracksDataException {

+        ITreeIndexFrame frame = leafFrameFactory.createFrame();

+        ITreeIndexMetaDataFrame metaFrame = freePageManager.getMetaDataFrameFactory().createFrame();

+        freePageManager.init(metaFrame, rootPage);

+

+        ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), true);

+        rootNode.acquireWriteLatch();

+        try {

+            frame.setPage(rootNode);

+            frame.initBuffer((byte) 0);

+        } finally {

+            rootNode.releaseWriteLatch();

+            bufferCache.unpin(rootNode);

+        }

+    }

+

+    public synchronized void activate() throws HyracksDataException {

+        if (isActivated) {

+            return;

+        }

+

+        boolean fileIsMapped = false;

+        synchronized (fileMapProvider) {

+            fileIsMapped = fileMapProvider.isMapped(file);

+            if (!fileIsMapped) {

+                bufferCache.createFile(file);

+            }

+            fileId = fileMapProvider.lookupFileId(file);

+            try {

+                // Also creates the file if it doesn't exist yet.

+                bufferCache.openFile(fileId);

+            } catch (HyracksDataException e) {

+                // Revert state of buffer cache since file failed to open.

+                if (!fileIsMapped) {

+                    bufferCache.deleteFile(fileId, false);

+                }

+                throw e;

+            }

+        }

+        freePageManager.open(fileId);

+

+        // TODO: Should probably have some way to check that the tree is physically consistent

+        // or that the file we just opened actually is a tree

+

+        isActivated = true;

+    }

+

+    public synchronized void deactivate() throws HyracksDataException {

+        if (!isActivated) {

+            return;

+        }

+

+        bufferCache.closeFile(fileId);

+        freePageManager.close();

+

+        isActivated = false;

+    }

+

+    public synchronized void destroy() throws HyracksDataException {

+        if (isActivated) {

+            throw new HyracksDataException("Failed to destroy the index since it is activated.");

+        }

+

+        file.delete();

+        if (fileId == -1) {

+            return;

+        }

+

+        bufferCache.deleteFile(fileId, false);

+        fileId = -1;

+    }

+

+    public synchronized void clear() throws HyracksDataException {

+        if (!isActivated) {

+            throw new HyracksDataException("Failed to clear the index since it is not activated.");

+        }

+        initEmptyTree();

+    }

+

+    public boolean isEmptyTree(ITreeIndexFrame frame) throws HyracksDataException {

+        ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false);

+        rootNode.acquireReadLatch();

+        try {

+            frame.setPage(rootNode);

+            if (frame.getLevel() == 0 && frame.getTupleCount() == 0) {

+                return true;

+            } else {

+                return false;

+            }

+        } finally {

+            rootNode.releaseReadLatch();

+            bufferCache.unpin(rootNode);

+        }

+    }

+

+    public byte getTreeHeight(ITreeIndexFrame frame) throws HyracksDataException {

+        ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false);

+        rootNode.acquireReadLatch();

+        try {

+            frame.setPage(rootNode);

+            return frame.getLevel();

+        } finally {

+            rootNode.releaseReadLatch();

+            bufferCache.unpin(rootNode);

+        }

+    }

+

+    public int getFileId() {

+        return fileId;

+    }

+

+    public FileReference getFileReference() {

+        return file;

+    }

+

+    public IBufferCache getBufferCache() {

+        return bufferCache;

+    }

+

+    public ITreeIndexFrameFactory getInteriorFrameFactory() {

+        return interiorFrameFactory;

+    }

+

+    public ITreeIndexFrameFactory getLeafFrameFactory() {

+        return leafFrameFactory;

+    }

+

+    public IBinaryComparatorFactory[] getComparatorFactories() {

+        return cmpFactories;

+    }

+

+    public IFreePageManager getFreePageManager() {

+        return freePageManager;

+    }

+

+    public int getRootPageId() {

+        return rootPage;

+    }

+

+    public int getFieldCount() {

+        return fieldCount;

+    }

+

+    public abstract class AbstractTreeIndexBulkLoader implements IIndexBulkLoader {

+        protected final MultiComparator cmp;

+        protected final int slotSize;

+        protected final int leafMaxBytes;

+        protected final int interiorMaxBytes;

+        protected final ArrayList<NodeFrontier> nodeFrontiers = new ArrayList<NodeFrontier>();

+        protected final ITreeIndexMetaDataFrame metaFrame;

+        protected final ITreeIndexTupleWriter tupleWriter;

+        protected ITreeIndexFrame leafFrame;

+        protected ITreeIndexFrame interiorFrame;

+

+        public AbstractTreeIndexBulkLoader(float fillFactor) throws TreeIndexException, HyracksDataException {

+            leafFrame = leafFrameFactory.createFrame();

+            interiorFrame = interiorFrameFactory.createFrame();

+            metaFrame = freePageManager.getMetaDataFrameFactory().createFrame();

+

+            if (!isEmptyTree(leafFrame)) {

+                throw new TreeIndexException("Cannot bulk-load a non-empty tree.");

+            }

+

+            this.cmp = MultiComparator.createIgnoreFieldLength(cmpFactories);

+

+            leafFrame.setMultiComparator(cmp);

+            interiorFrame.setMultiComparator(cmp);

+

+            tupleWriter = leafFrame.getTupleWriter();

+

+            NodeFrontier leafFrontier = new NodeFrontier(leafFrame.createTupleReference());

+            leafFrontier.pageId = freePageManager.getFreePage(metaFrame);

+            leafFrontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId), true);

+            leafFrontier.page.acquireWriteLatch();

+

+            interiorFrame.setPage(leafFrontier.page);

+            interiorFrame.initBuffer((byte) 0);

+            interiorMaxBytes = (int) ((float) interiorFrame.getBuffer().capacity() * fillFactor);

+

+            leafFrame.setPage(leafFrontier.page);

+            leafFrame.initBuffer((byte) 0);

+            leafMaxBytes = (int) ((float) leafFrame.getBuffer().capacity() * fillFactor);

+            slotSize = leafFrame.getSlotSize();

+

+            nodeFrontiers.add(leafFrontier);

+        }

+

+        public abstract void add(ITupleReference tuple) throws IndexException, HyracksDataException;

+

+        protected void handleException() throws HyracksDataException {

+            // Unlatch and unpin pages.

+            for (NodeFrontier nodeFrontier : nodeFrontiers) {

+                nodeFrontier.page.releaseWriteLatch();

+                bufferCache.unpin(nodeFrontier.page);

+            }

+        }

+

+        @Override

+        public void end() throws HyracksDataException {

+            // copy the root generated from the bulk-load to *the* root page location

+            ICachedPage newRoot = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), true);

+            newRoot.acquireWriteLatch();

+            NodeFrontier lastNodeFrontier = nodeFrontiers.get(nodeFrontiers.size() - 1);

+            try {

+                System.arraycopy(lastNodeFrontier.page.getBuffer().array(), 0, newRoot.getBuffer().array(), 0,

+                        lastNodeFrontier.page.getBuffer().capacity());

+            } finally {

+                newRoot.releaseWriteLatch();

+                bufferCache.unpin(newRoot);

+

+                // register old root as a free page

+                freePageManager.addFreePage(metaFrame, lastNodeFrontier.pageId);

+

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

+                    nodeFrontiers.get(i).page.releaseWriteLatch();

+                    bufferCache.unpin(nodeFrontiers.get(i).page);

+                }

+            }

+        }

+

+        protected void addLevel() throws HyracksDataException {

+            NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());

+            frontier.pageId = freePageManager.getFreePage(metaFrame);

+            frontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, frontier.pageId), true);

+            frontier.page.acquireWriteLatch();

+            frontier.lastTuple.setFieldCount(cmp.getKeyFieldCount());

+            interiorFrame.setPage(frontier.page);

+            interiorFrame.initBuffer((byte) nodeFrontiers.size());

+            nodeFrontiers.add(frontier);

+        }

+    }

+

+    public class TreeIndexInsertBulkLoader implements IIndexBulkLoader {

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

+                NoOpOperationCallback.INSTANCE);

+

+        @Override

+        public void add(ITupleReference tuple) throws HyracksDataException {

+            try {

+                accessor.insert(tuple);

+            } catch (IndexException e) {

+                throw new HyracksDataException(e);

+            }

+        }

+

+        @Override

+        public void end() throws HyracksDataException {

+            // do nothing

+        }

+

+    }

+

+    @Override

+    public long getMemoryAllocationSize() {

+        return 0;

+    }

+}

diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallback.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallback.java
index 828dd81..6fc8dcf 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallback.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallback.java
@@ -16,26 +16,37 @@
 package edu.uci.ics.hyracks.storage.am.common.impls;
 
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
 
 /**
- * Dummy operation callback that simply does nothing. Mainly, intended to be
- * used in non-transaction access method testing.
+ * Dummy operation callback that simply does nothing.
  */
-public class NoOpOperationCallback implements IOperationCallback {
+public enum NoOpOperationCallback implements IModificationOperationCallback, ISearchOperationCallback {
+    INSTANCE;
 
-    public static IOperationCallback INSTANCE = new NoOpOperationCallback();
-    
-    private NoOpOperationCallback() {
-    }
-    
     @Override
-    public void pre(ITupleReference tuple) {
+    public boolean proceed(ITupleReference tuple) {
+        return true;
+    }
+
+    @Override
+    public void reconcile(ITupleReference tuple) {
         // Do nothing.
     }
 
     @Override
-    public void post(ITupleReference tuple) {
+    public void before(ITupleReference tuple) {
+        // Do nothing.        
+    }
+
+    @Override
+    public void found(ITupleReference before, ITupleReference after) {
+        // Do nothing.        
+    }
+
+    @Override
+    public void cancel(ITupleReference tuple) {
         // Do nothing.
     }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
new file mode 100644
index 0000000..4c641a6
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.hyracks.storage.am.common.impls;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+
+/**
+ * Dummy NoOp callback factory used primarily for testing. Always returns the {@link NoOpOperationCallback} instance.
+ * Implemented as an enum to preserve singleton model while being serializable
+ */
+public enum NoOpOperationCallbackFactory implements ISearchOperationCallbackFactory,
+        IModificationOperationCallbackFactory {
+    INSTANCE;
+
+    @Override
+    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource, IHyracksTaskContext ctx) {
+        return NoOpOperationCallback.INSTANCE;
+    }
+
+    @Override
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx) {
+        return NoOpOperationCallback.INSTANCE;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallbackProvider.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallbackProvider.java
deleted file mode 100644
index 55dfb74e..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NoOpOperationCallbackProvider.java
+++ /dev/null
@@ -1,19 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.impls;
-
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
-
-/**
- * Dummy NoOp callback provider used primarily for testing. Always returns the 
- * {@link NoOpOperationCallback} instance. 
- *
- * Implemented as an enum to preserve singleton model while being serializable
- */
-public enum NoOpOperationCallbackProvider implements IOperationCallbackProvider {
-    INSTANCE;
-
-    @Override
-    public IOperationCallback getOperationCallback() {
-        return NoOpOperationCallback.INSTANCE;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NodeFrontier.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NodeFrontier.java
new file mode 100644
index 0000000..56f5fdb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/NodeFrontier.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 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.common.impls;
+
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public class NodeFrontier {
+    public ICachedPage page;
+    public int pageId;
+    public ITreeIndexTupleReference lastTuple;
+
+    public NodeFrontier(ITreeIndexTupleReference lastTuple) {
+        this.lastTuple = lastTuple;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java
deleted file mode 100644
index ea4c105..0000000
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.impls;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
-import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
-
-public class TreeDiskOrderScanCursor implements ITreeIndexCursor {
-
-	private int tupleIndex = 0;
-	private int fileId = -1;
-	private int currentPageId = -1;
-	private int maxPageId = -1;
-	private ICachedPage page = null;	
-	private IBufferCache bufferCache = null;
-	
-	private final ITreeIndexFrame frame;
-	private final ITreeIndexTupleReference frameTuple;
-	
-	public TreeDiskOrderScanCursor(ITreeIndexFrame frame) {
-		this.frame = frame;		
-		this.frameTuple = frame.createTupleReference();
-	}
-
-	@Override
-	public void close() throws HyracksDataException {
-		page.releaseReadLatch();
-		bufferCache.unpin(page);
-		page = null;
-	}
-
-	@Override
-	public ITreeIndexTupleReference getTuple() {
-		return frameTuple;
-	}
-
-	@Override
-	public ICachedPage getPage() {
-		return page;
-	}
-
-	private boolean positionToNextLeaf(boolean skipCurrent)
-			throws HyracksDataException {
-		while ((frame.getLevel() != 0 || skipCurrent || frame.getTupleCount() == 0) && (currentPageId <= maxPageId)) {
-			currentPageId++;
-
-			page.releaseReadLatch();
-            bufferCache.unpin(page);
-			
-			ICachedPage nextPage = bufferCache.pin(
-					BufferedFileHandle.getDiskPageId(fileId, currentPageId),
-					false);
-			nextPage.acquireReadLatch();
-
-			page = nextPage;
-			frame.setPage(page);
-			tupleIndex = 0;
-			skipCurrent = false;
-		}
-		if (currentPageId <= maxPageId) {
-			return true;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public boolean hasNext() throws HyracksDataException {		
-		if (currentPageId > maxPageId) {
-			return false;
-		}
-		if (tupleIndex >= frame.getTupleCount()) {
-			boolean nextLeafExists = positionToNextLeaf(true);
-			if (nextLeafExists) {
-				frameTuple.resetByTupleIndex(frame, tupleIndex);
-				return true;
-			} else {
-				return false;
-			}
-		}		
-		frameTuple.resetByTupleIndex(frame, tupleIndex);		
-		return true;
-	}
-
-	@Override
-	public void next() throws HyracksDataException {
-		tupleIndex++;
-	}
-
-	@Override
-	public void open(ICursorInitialState initialState,
-			ISearchPredicate searchPred) throws HyracksDataException {
-		// in case open is called multiple times without closing
-		if (page != null) {
-			page.releaseReadLatch();
-			bufferCache.unpin(page);
-		}
-		page = initialState.getPage();
-		tupleIndex = 0;		
-		frame.setPage(page);
-		positionToNextLeaf(false);
-	}
-
-	@Override
-	public void reset() {
-		tupleIndex = 0;
-		currentPageId = -1;
-		maxPageId = -1;
-		page = null;
-	}
-
-	@Override
-	public void setBufferCache(IBufferCache bufferCache) {
-		this.bufferCache = bufferCache;
-	}
-
-	@Override
-	public void setFileId(int fileId) {
-		this.fileId = fileId;
-	}
-
-	public void setCurrentPageId(int currentPageId) {
-		this.currentPageId = currentPageId;
-	}
-
-	public void setMaxPageId(int maxPageId) {
-		this.maxPageId = maxPageId;
-	}
-
-	@Override
-	public boolean exclusiveLatchNodes() {
-		return false;
-	}
-}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
new file mode 100644
index 0000000..738d987
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
@@ -0,0 +1,150 @@
+/*
+ * Copyright 2009-2010 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.common.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+
+public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor {
+
+    private int tupleIndex = 0;
+    private int fileId = -1;
+    private int currentPageId = -1;
+    private int maxPageId = -1;
+    private ICachedPage page = null;
+    private IBufferCache bufferCache = null;
+
+    private final ITreeIndexFrame frame;
+    private final ITreeIndexTupleReference frameTuple;
+
+    public TreeIndexDiskOrderScanCursor(ITreeIndexFrame frame) {
+        this.frame = frame;
+        this.frameTuple = frame.createTupleReference();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        page.releaseReadLatch();
+        bufferCache.unpin(page);
+        page = null;
+    }
+
+    @Override
+    public ITreeIndexTupleReference getTuple() {
+        return frameTuple;
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        return page;
+    }
+
+    private boolean positionToNextLeaf(boolean skipCurrent) throws HyracksDataException {
+        while ((frame.getLevel() != 0 || skipCurrent || frame.getTupleCount() == 0) && (currentPageId <= maxPageId)) {
+            currentPageId++;
+
+            page.releaseReadLatch();
+            bufferCache.unpin(page);
+
+            ICachedPage nextPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
+            nextPage.acquireReadLatch();
+
+            page = nextPage;
+            frame.setPage(page);
+            tupleIndex = 0;
+            skipCurrent = false;
+        }
+        if (currentPageId <= maxPageId) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException {
+        if (currentPageId > maxPageId) {
+            return false;
+        }
+        if (tupleIndex >= frame.getTupleCount()) {
+            boolean nextLeafExists = positionToNextLeaf(true);
+            if (nextLeafExists) {
+                frameTuple.resetByTupleIndex(frame, tupleIndex);
+                return true;
+            } else {
+                return false;
+            }
+        }
+        frameTuple.resetByTupleIndex(frame, tupleIndex);
+        return true;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        tupleIndex++;
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        // in case open is called multiple times without closing
+        if (page != null) {
+            page.releaseReadLatch();
+            bufferCache.unpin(page);
+        }
+        page = initialState.getPage();
+        tupleIndex = 0;
+        frame.setPage(page);
+        positionToNextLeaf(false);
+    }
+
+    @Override
+    public void reset() {
+        tupleIndex = 0;
+        currentPageId = -1;
+        maxPageId = -1;
+        page = null;
+    }
+
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        this.bufferCache = bufferCache;
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        this.fileId = fileId;
+    }
+
+    public void setCurrentPageId(int currentPageId) {
+        this.currentPageId = currentPageId;
+    }
+
+    public void setMaxPageId(int maxPageId) {
+        this.maxPageId = maxPageId;
+    }
+
+    @Override
+    public boolean exclusiveLatchNodes() {
+        return false;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/DoubleArrayList.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/DoubleArrayList.java
new file mode 100644
index 0000000..318a102
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/DoubleArrayList.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 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.common.ophelpers;
+
+public class DoubleArrayList {
+    private double[] data;
+    private int size;
+    private int first;
+    private final int growth;
+
+    public DoubleArrayList(int initialCapacity, int growth) {
+        data = new double[initialCapacity];
+        size = 0;
+        first = 0;
+        this.growth = growth;
+    }
+
+    public int size() {
+        return size;
+    }
+
+    public int first() {
+        return first;
+    }
+
+    public void add(double i) {
+        if (size == data.length) {
+            double[] newData = new double[data.length + growth];
+            System.arraycopy(data, 0, newData, 0, data.length);
+            data = newData;
+        }
+
+        data[size++] = i;
+    }
+
+    public void addFirst(double i) {
+        double[] newData = new double[data.length + 1];
+        System.arraycopy(data, 0, newData, 0, first);
+        System.arraycopy(data, first, newData, first + 1, size - first);
+        data = newData;
+        data[first] = i;
+        size++;
+    }
+
+    public void removeLast() {
+        if (size > 0)
+            size--;
+    }
+
+    // WARNING: caller is responsible for checking size > 0
+    public double getLast() {
+        return data[size - 1];
+    }
+
+    public double get(int i) {
+        return data[i];
+    }
+
+    // WARNING: caller is responsible for checking i < size
+    public void set(int i, int value) {
+        data[i] = value;
+
+    }
+
+    public double getFirst() {
+        return data[first];
+    }
+
+    public void moveFirst() {
+        first++;
+    }
+
+    public void clear() {
+        size = 0;
+        first = 0;
+    }
+
+    public boolean isLast() {
+        return size == first;
+    }
+
+    public boolean isEmpty() {
+        return size == 0;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/FieldLengthIgnoringMultiComparator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/FieldLengthIgnoringMultiComparator.java
new file mode 100644
index 0000000..4e43a34
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/FieldLengthIgnoringMultiComparator.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2012 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.common.ophelpers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * MultiComparator that always passes 0 as a tuple's field length. This may speed up comparisons.
+ */
+public class FieldLengthIgnoringMultiComparator extends MultiComparator {
+
+    public FieldLengthIgnoringMultiComparator(IBinaryComparator[] cmps) {
+        super(cmps);
+    }
+
+    @Override
+    public int compare(ITupleReference tupleA, ITupleReference tupleB) {
+        for (int i = 0; i < cmps.length; i++) {
+            int cmp = cmps[i].compare(tupleA.getFieldData(i), tupleA.getFieldStart(i), 0, tupleB.getFieldData(i),
+                    tupleB.getFieldStart(i), 0);
+            if (cmp != 0) {
+                return cmp;
+            }
+        }
+        return 0;
+    }
+
+    @Override
+    public int selectiveFieldCompare(ITupleReference tupleA, ITupleReference tupleB, int[] fields) {
+        for (int j = 0; j < cmps.length; j++) {
+            int i = fields[j];
+            int cmp = cmps[j].compare(tupleA.getFieldData(i), tupleA.getFieldStart(i), 0, tupleB.getFieldData(i),
+                    tupleB.getFieldStart(i), 0);
+            if (cmp != 0) {
+                return cmp;
+            }
+        }
+        return 0;
+    }
+
+    @Override
+    public int compare(ITupleReference tupleA, ITupleReference tupleB, int startFieldIndex) {
+        for (int i = 0; i < cmps.length; i++) {
+            int ix = startFieldIndex + i;
+            int cmp = cmps[i].compare(tupleA.getFieldData(ix), tupleA.getFieldStart(ix), 0, tupleB.getFieldData(ix),
+                    tupleB.getFieldStart(ix), 0);
+            if (cmp != 0) {
+                return cmp;
+            }
+        }
+        return 0;
+    }
+
+    @Override
+    public int fieldRangeCompare(ITupleReference tupleA, ITupleReference tupleB, int startFieldIndex, int numFields) {
+        for (int i = startFieldIndex; i < startFieldIndex + numFields; i++) {
+            int cmp = cmps[i].compare(tupleA.getFieldData(i), tupleA.getFieldStart(i), 0, tupleB.getFieldData(i),
+                    tupleB.getFieldStart(i), 0);
+            if (cmp != 0) {
+                return cmp;
+            }
+        }
+        return 0;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/FieldLengthIgnoringSingleComparator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/FieldLengthIgnoringSingleComparator.java
new file mode 100644
index 0000000..d35297a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/FieldLengthIgnoringSingleComparator.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2012 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.common.ophelpers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * MultiComparator optimized for the special case where there is only a single comparator.
+ * Further speeds up comparisons by always passing 0 as the field's length.
+ */
+public class FieldLengthIgnoringSingleComparator extends MultiComparator {
+    private final IBinaryComparator cmp;
+
+    protected FieldLengthIgnoringSingleComparator(IBinaryComparator[] cmps) {
+        super(cmps);
+        this.cmp = cmps[0];
+    }
+
+    public int compare(ITupleReference tupleA, ITupleReference tupleB) {
+        return cmp.compare(tupleA.getFieldData(0), tupleA.getFieldStart(0), 0, tupleB.getFieldData(0),
+                tupleB.getFieldStart(0), 0);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/IndexOperation.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/IndexOperation.java
new file mode 100644
index 0000000..1885348
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/IndexOperation.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 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.common.ophelpers;
+
+public enum IndexOperation {
+    INSERT,
+    DELETE,
+    UPDATE,
+    UPSERT,
+    SEARCH,
+    DISKORDERSCAN,
+    PHYSICALDELETE,
+    NOOP,
+    MERGE,
+    FLUSH
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/MultiComparator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/MultiComparator.java
index c653c9a..567973d 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/MultiComparator.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/MultiComparator.java
@@ -21,70 +21,131 @@
 
 public class MultiComparator {
 
-	private final IBinaryComparator[] cmps;
+    protected final IBinaryComparator[] cmps;
 
-	public MultiComparator(IBinaryComparator[] cmps) {
-		this.cmps = cmps;
-	}
+    public MultiComparator(IBinaryComparator[] cmps) {
+        this.cmps = cmps;
+    }
 
-	public int compare(ITupleReference tupleA, ITupleReference tupleB) {
-		for (int i = 0; i < cmps.length; i++) {
-			int cmp = cmps[i].compare(tupleA.getFieldData(i),
-					tupleA.getFieldStart(i), tupleA.getFieldLength(i),
-					tupleB.getFieldData(i), tupleB.getFieldStart(i),
-					tupleB.getFieldLength(i));
-			if (cmp < 0)
-				return -1;
-			else if (cmp > 0)
-				return 1;
-		}
-		return 0;
-	}
+    public int compare(ITupleReference tupleA, ITupleReference tupleB) {
+        for (int i = 0; i < cmps.length; i++) {
+            int cmp = cmps[i].compare(tupleA.getFieldData(i), tupleA.getFieldStart(i), tupleA.getFieldLength(i),
+                    tupleB.getFieldData(i), tupleB.getFieldStart(i), tupleB.getFieldLength(i));
+            if (cmp != 0) {
+                return cmp;
+            }
+        }
+        return 0;
+    }
 
-	public int fieldRangeCompare(ITupleReference tupleA,
-			ITupleReference tupleB, int startFieldIndex, int numFields) {
-		for (int i = startFieldIndex; i < startFieldIndex + numFields; i++) {
-			int cmp = cmps[i].compare(tupleA.getFieldData(i),
-					tupleA.getFieldStart(i), tupleA.getFieldLength(i),
-					tupleB.getFieldData(i), tupleB.getFieldStart(i),
-					tupleB.getFieldLength(i));
-			if (cmp < 0)
-				return -1;
-			else if (cmp > 0)
-				return 1;
-		}
-		return 0;
-	}
-	
-	public int compare(ITupleReference tupleA,
-			ITupleReference tupleB, int startFieldIndex) {
-		for (int i = 0; i < cmps.length; i++) {
-			int ix = startFieldIndex + i;
-			int cmp = cmps[i].compare(tupleA.getFieldData(ix),
-					tupleA.getFieldStart(ix), tupleA.getFieldLength(ix),
-					tupleB.getFieldData(ix), tupleB.getFieldStart(ix),
-					tupleB.getFieldLength(ix));
-			if (cmp < 0)
-				return -1;
-			else if (cmp > 0)
-				return 1;
-		}
-		return 0;
-	}
+    public int selectiveFieldCompare(ITupleReference tupleA, ITupleReference tupleB, int[] fields) {
+        for (int j = 0; j < cmps.length; j++) {
+            int i = fields[j];
+            int cmp = cmps[j].compare(tupleA.getFieldData(i), tupleA.getFieldStart(i), tupleA.getFieldLength(i),
+                    tupleB.getFieldData(i), tupleB.getFieldStart(i), tupleB.getFieldLength(i));
+            if (cmp != 0) {
+                return cmp;
+            }
+        }
+        return 0;
+    }
 
-	public IBinaryComparator[] getComparators() {
-		return cmps;
-	}
+    public int compare(ITupleReference tupleA, ITupleReference tupleB, int startFieldIndex) {
+        for (int i = 0; i < cmps.length; i++) {
+            int ix = startFieldIndex + i;
+            int cmp = cmps[i].compare(tupleA.getFieldData(ix), tupleA.getFieldStart(ix), tupleA.getFieldLength(ix),
+                    tupleB.getFieldData(ix), tupleB.getFieldStart(ix), tupleB.getFieldLength(ix));
+            if (cmp != 0) {
+                return cmp;
+            }
+        }
+        return 0;
+    }
+
+    public int fieldRangeCompare(ITupleReference tupleA, ITupleReference tupleB, int startFieldIndex, int numFields) {
+        for (int i = startFieldIndex; i < startFieldIndex + numFields; i++) {
+            int cmp = cmps[i].compare(tupleA.getFieldData(i), tupleA.getFieldStart(i), tupleA.getFieldLength(i),
+                    tupleB.getFieldData(i), tupleB.getFieldStart(i), tupleB.getFieldLength(i));
+            if (cmp != 0) {
+                return cmp;
+            }
+        }
+        return 0;
+    }
+
+    public IBinaryComparator[] getComparators() {
+        return cmps;
+    }
 
     public int getKeyFieldCount() {
-		return cmps.length;
-	}
-    
+        return cmps.length;
+    }
+
     public static MultiComparator create(IBinaryComparatorFactory[] cmpFactories) {
         IBinaryComparator[] cmps = new IBinaryComparator[cmpFactories.length];
         for (int i = 0; i < cmpFactories.length; i++) {
             cmps[i] = cmpFactories[i].createBinaryComparator();
         }
-        return new MultiComparator(cmps);
+        if (cmps.length == 1) {
+            return new SingleComparator(cmps);
+        } else {
+            return new MultiComparator(cmps);
+        }
+    }
+
+    public static MultiComparator createIgnoreFieldLength(IBinaryComparatorFactory[] cmpFactories) {
+        IBinaryComparator[] cmps = new IBinaryComparator[cmpFactories.length];
+        for (int i = 0; i < cmpFactories.length; i++) {
+            cmps[i] = cmpFactories[i].createBinaryComparator();
+        }
+        if (cmps.length == 1) {
+            return new FieldLengthIgnoringSingleComparator(cmps);
+        } else {
+            return new FieldLengthIgnoringMultiComparator(cmps);
+        }
+    }
+
+    public static MultiComparator createIgnoreFieldLength(IBinaryComparatorFactory[] cmpFactories, int startIndex,
+            int numCmps) {
+        IBinaryComparator[] cmps = new IBinaryComparator[numCmps];
+        for (int i = startIndex; i < startIndex + numCmps; i++) {
+            cmps[i] = cmpFactories[i].createBinaryComparator();
+        }
+        if (cmps.length == 1) {
+            return new FieldLengthIgnoringSingleComparator(cmps);
+        } else {
+            return new FieldLengthIgnoringMultiComparator(cmps);
+        }
+    }
+
+    public static MultiComparator create(IBinaryComparatorFactory[] cmpFactories, int startIndex, int numCmps) {
+        IBinaryComparator[] cmps = new IBinaryComparator[numCmps];
+        for (int i = startIndex; i < startIndex + numCmps; i++) {
+            cmps[i] = cmpFactories[i].createBinaryComparator();
+        }
+        if (cmps.length == 1) {
+            return new SingleComparator(cmps);
+        } else {
+            return new MultiComparator(cmps);
+        }
+    }
+
+    public static MultiComparator create(IBinaryComparatorFactory[]... cmpFactories) {
+        int size = 0;
+        for (int i = 0; i < cmpFactories.length; i++) {
+            size += cmpFactories[i].length;
+        }
+        IBinaryComparator[] cmps = new IBinaryComparator[size];
+        int x = 0;
+        for (int i = 0; i < cmpFactories.length; i++) {
+            for (int j = 0; j < cmpFactories[i].length; j++) {
+                cmps[x++] = cmpFactories[i][j].createBinaryComparator();
+            }
+        }
+        if (cmps.length == 1) {
+            return new SingleComparator(cmps);
+        } else {
+            return new MultiComparator(cmps);
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/SingleComparator.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/SingleComparator.java
new file mode 100644
index 0000000..c8841f3
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/SingleComparator.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 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.common.ophelpers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * MultiComparator optimized for the special case where there is only a single comparator.
+ */
+public class SingleComparator extends MultiComparator {
+
+    private final IBinaryComparator cmp;
+
+    protected SingleComparator(IBinaryComparator[] cmps) {
+        super(cmps);
+        this.cmp = cmps[0];
+    }
+
+    public int compare(ITupleReference tupleA, ITupleReference tupleB) {
+        return cmp.compare(tupleA.getFieldData(0), tupleA.getFieldStart(0), tupleA.getFieldLength(0),
+                tupleB.getFieldData(0), tupleB.getFieldStart(0), tupleB.getFieldLength(0));
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
new file mode 100644
index 0000000..c5662b1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2009-2012 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.common.tuples;
+
+import java.util.Arrays;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * WARNING: getFieldData(), getFieldStart() and getFieldLength() are log and not constant time.
+ */
+public class ConcatenatingTupleReference implements ITupleReference {
+
+    private final ITupleReference[] tuples;
+    private final int[] fieldCounts;
+    private int numTuples;
+    private int totalFieldCount;
+    
+    public ConcatenatingTupleReference(int maxTuples) {
+        tuples = new ITupleReference[maxTuples];
+        fieldCounts = new int[maxTuples];
+        reset();        
+    }
+    
+    public void reset() {
+        numTuples = 0;
+        totalFieldCount = 0;
+    }
+    
+    public void addTuple(ITupleReference tuple) {
+        tuples[numTuples] = tuple;
+        totalFieldCount += tuple.getFieldCount();
+        if (numTuples > 0) {
+            fieldCounts[numTuples] = fieldCounts[numTuples - 1] + tuple.getFieldCount();
+        } else {
+            fieldCounts[numTuples] = tuple.getFieldCount();
+        }
+        ++numTuples;
+    }
+    
+    public void removeLastTuple() {
+        if (numTuples > 0) {
+            ITupleReference lastTuple = tuples[--numTuples];
+            totalFieldCount -= lastTuple.getFieldCount();
+        }
+    }
+    
+    public int getNumTuples() {
+        return numTuples;
+    }
+    
+    public boolean hasMaxTuples() {
+        return numTuples == tuples.length;
+    }
+    
+    @Override
+    public int getFieldCount() {
+        return totalFieldCount;
+    }
+
+    @Override
+    public byte[] getFieldData(int fIdx) {
+        int tupleIndex = getTupleIndex(fIdx);
+        int fieldIndex = getFieldIndex(tupleIndex, fIdx);
+        return tuples[tupleIndex].getFieldData(fieldIndex);
+    }
+
+    @Override
+    public int getFieldStart(int fIdx) {
+        int tupleIndex = getTupleIndex(fIdx);
+        int fieldIndex = getFieldIndex(tupleIndex, fIdx);
+        return tuples[tupleIndex].getFieldStart(fieldIndex);
+    }
+
+    @Override
+    public int getFieldLength(int fIdx) {
+        int tupleIndex = getTupleIndex(fIdx);
+        int fieldIndex = getFieldIndex(tupleIndex, fIdx);
+        return tuples[tupleIndex].getFieldLength(fieldIndex);
+    }
+    
+    private int getTupleIndex(int fIdx) {
+        int tupleIndex = Arrays.binarySearch(fieldCounts, 0, numTuples - 1, fIdx);
+        if (tupleIndex < 0) {
+            tupleIndex = -tupleIndex - 1;
+        } else {
+            ++tupleIndex;
+        }
+        return tupleIndex;
+    }
+    
+    private int getFieldIndex(int tupleIndex, int fIdx) {
+        int fieldIndex = -1;
+        if (tupleIndex > 0) {
+            fieldIndex = fIdx - fieldCounts[tupleIndex - 1]; 
+        } else {
+            fieldIndex = fIdx;
+        }
+        return fieldIndex;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/PermutingFrameTupleReference.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/PermutingFrameTupleReference.java
new file mode 100644
index 0000000..5f97f04
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/PermutingFrameTupleReference.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 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.common.tuples;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class PermutingFrameTupleReference implements IFrameTupleReference {
+	private IFrameTupleAccessor fta;
+	private int tIndex;
+	private int[] fieldPermutation;
+
+	public void setFieldPermutation(int[] fieldPermutation) {
+		this.fieldPermutation = fieldPermutation;
+	}
+
+	public void reset(IFrameTupleAccessor fta, int tIndex) {
+		this.fta = fta;
+		this.tIndex = tIndex;
+	}
+
+	@Override
+	public IFrameTupleAccessor getFrameTupleAccessor() {
+		return fta;
+	}
+
+	@Override
+	public int getTupleIndex() {
+		return tIndex;
+	}
+
+	@Override
+	public int getFieldCount() {
+		return fieldPermutation.length;
+	}
+
+	@Override
+	public byte[] getFieldData(int fIdx) {
+		return fta.getBuffer().array();
+	}
+
+	@Override
+	public int getFieldStart(int fIdx) {
+		return fta.getTupleStartOffset(tIndex) + fta.getFieldSlotsLength()
+				+ fta.getFieldStartOffset(tIndex, fieldPermutation[fIdx]);
+	}
+
+	@Override
+	public int getFieldLength(int fIdx) {
+		return fta.getFieldLength(tIndex, fieldPermutation[fIdx]);
+	}
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/PermutingTupleReference.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/PermutingTupleReference.java
new file mode 100644
index 0000000..0272ff6
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/PermutingTupleReference.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2009-2012 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.common.tuples;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class PermutingTupleReference implements ITupleReference {
+
+    private final int[] fieldPermutation;
+    private ITupleReference sourceTuple;
+    
+    public PermutingTupleReference(int[] fieldPermutation) {
+        this.fieldPermutation = fieldPermutation;
+    }
+    
+    public void reset(ITupleReference sourceTuple) {
+        this.sourceTuple = sourceTuple;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return fieldPermutation.length;
+    }
+
+    @Override
+    public byte[] getFieldData(int fIdx) {
+        return sourceTuple.getFieldData(fieldPermutation[fIdx]);
+    }
+
+    @Override
+    public int getFieldStart(int fIdx) {
+        return sourceTuple.getFieldStart(fieldPermutation[fIdx]);
+    }
+
+    @Override
+    public int getFieldLength(int fIdx) {
+        return sourceTuple.getFieldLength(fieldPermutation[fIdx]);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/SimpleTupleWriter.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/SimpleTupleWriter.java
index f5ec5f3..8c41dd3 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/SimpleTupleWriter.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/SimpleTupleWriter.java
@@ -23,12 +23,12 @@
 
 public class SimpleTupleWriter implements ITreeIndexTupleWriter {
 
-	// Write short in little endian to target byte array at given offset.
-	private static void writeShortL(short s, byte[] buf, int targetOff) {
-		buf[targetOff] = (byte)(s >> 8);
-		buf[targetOff + 1] = (byte)(s >> 0);
-	}
-	
+    // Write short in little endian to target byte array at given offset.
+    private static void writeShortL(short s, byte[] buf, int targetOff) {
+        buf[targetOff] = (byte) (s >> 8);
+        buf[targetOff + 1] = (byte) (s >> 0);
+    }
+
     @Override
     public int bytesRequired(ITupleReference tuple) {
         int bytes = getNullFlagsBytes(tuple) + getFieldSlotsBytes(tuple);
@@ -56,10 +56,10 @@
     public int writeTuple(ITupleReference tuple, ByteBuffer targetBuf, int targetOff) {
         return writeTuple(tuple, targetBuf.array(), targetOff);
     }
-    
+
     @Override
-	public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
-    	int runner = targetOff;
+    public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
+        int runner = targetOff;
         int nullFlagsBytes = getNullFlagsBytes(tuple);
         int fieldSlotsBytes = getFieldSlotsBytes(tuple);
         for (int i = 0; i < nullFlagsBytes; i++) {
@@ -68,18 +68,16 @@
         runner += fieldSlotsBytes;
         int fieldEndOff = 0;
         for (int i = 0; i < tuple.getFieldCount(); i++) {
-            System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner,
-                    tuple.getFieldLength(i));
+            System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
             fieldEndOff += tuple.getFieldLength(i);
             runner += tuple.getFieldLength(i);
             writeShortL((short) fieldEndOff, targetBuf, targetOff + nullFlagsBytes + i * 2);
         }
         return runner - targetOff;
-	}
+    }
 
     @Override
-    public int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf,
-            int targetOff) {
+    public int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf, int targetOff) {
         int runner = targetOff;
         int nullFlagsBytes = getNullFlagsBytes(tuple, startField, numFields);
         for (int i = 0; i < nullFlagsBytes; i++) {
@@ -90,10 +88,9 @@
         int fieldEndOff = 0;
         int fieldCounter = 0;
         for (int i = startField; i < startField + numFields; i++) {
-            System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner,
-                    tuple.getFieldLength(i));
+            System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
             fieldEndOff += tuple.getFieldLength(i);
-            runner += tuple.getFieldLength(i);            
+            runner += tuple.getFieldLength(i);
             writeShortL((short) fieldEndOff, targetBuf, targetOff + nullFlagsBytes + fieldCounter * 2);
             fieldCounter++;
         }
@@ -115,5 +112,10 @@
 
     protected int getFieldSlotsBytes(ITupleReference tuple, int startField, int numFields) {
         return numFields * 2;
-    }	
+    }
+
+    @Override
+    public int getCopySpaceRequired(ITupleReference tuple) {
+        return bytesRequired(tuple);
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
index 9730346..1e12bea 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/tuples/TypeAwareTupleWriter.java
@@ -87,8 +87,7 @@
     }
 
     @Override
-    public int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf,
-            int targetOff) {
+    public int writeTupleFields(ITupleReference tuple, int startField, int numFields, byte[] targetBuf, int targetOff) {
         int runner = targetOff;
         int nullFlagsBytes = getNullFlagsBytes(numFields);
         // write null indicator bits
@@ -106,8 +105,7 @@
         runner = encDec.getPos();
 
         for (int i = startField; i < startField + numFields; i++) {
-            System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner,
-                    tuple.getFieldLength(i));
+            System.arraycopy(tuple.getFieldData(i), tuple.getFieldStart(i), targetBuf, runner, tuple.getFieldLength(i));
             runner += tuple.getFieldLength(i);
         }
 
@@ -149,4 +147,9 @@
     public void setTypeTraits(ITypeTraits[] typeTraits) {
         this.typeTraits = typeTraits;
     }
+
+    @Override
+    public int getCopySpaceRequired(ITupleReference tuple) {
+        return bytesRequired(tuple);
+    }
 }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/HashMultiSet.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/HashMultiSet.java
new file mode 100644
index 0000000..e4ccdcb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/HashMultiSet.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2009-2010 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.common.util;
+
+import java.util.AbstractCollection;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Quick and dirty implementation of a HashMultiSet backed by a HashMap.
+ * It only implements a minimal subset of the collection interface to make our tests work.
+ */
+public class HashMultiSet<E> extends AbstractCollection<E> {
+
+    private final Map<E, List<E>> map = new HashMap<E, List<E>>(); 
+    private int size = 0;
+    
+    @Override
+    public boolean add(E e) {
+        List<E> list = map.get(e);
+        if (list == null) {
+            list = new ArrayList<E>();
+            map.put(e, list);
+        }
+        list.add(e);
+        size++;
+        return true;
+    }
+    
+    @Override
+    public boolean contains(Object o) {
+        return map.containsKey(o);
+    }
+    
+    @Override
+    public boolean remove(Object o) {
+        List<E> list = map.get(o);
+        if (list == null) {
+            return false;            
+        }
+        list.remove(list.size() - 1);
+        if (list.isEmpty()) {
+            map.remove(o);
+        }
+        size--;
+        return true;
+    }
+    
+    @Override
+    public Iterator<E> iterator() {
+        return new HashMultiSetIterator();
+    }
+
+    @Override
+    public int size() {
+        return size;
+    }
+    
+    @Override
+    public void clear() {
+        map.clear();
+        size = 0;
+    }
+    
+    private class HashMultiSetIterator implements Iterator<E> {
+
+        private Iterator<Map.Entry<E, List<E>>> mapIter;
+        private Iterator<E> listIter;
+        
+        public HashMultiSetIterator() {
+            mapIter = map.entrySet().iterator();
+        }
+        
+        @Override
+        public boolean hasNext() {
+            if (mapIter.hasNext() || (listIter != null && listIter.hasNext())) {
+                return true;
+            }
+            return false;
+        }
+
+        @Override
+        public E next() {
+            if (listIter == null || (listIter != null && !listIter.hasNext())) {
+                Map.Entry<E, List<E>> entry = mapIter.next();
+                listIter = entry.getValue().iterator();
+                return listIter.next();
+            }
+            return listIter.next();
+        }
+
+        @Override
+        public void remove() {
+            throw new IllegalStateException("Not implemented");
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/resources/dist.all.first.cleaned b/hyracks/hyracks-storage-am-common/src/main/resources/dist.all.first.cleaned
new file mode 100755
index 0000000..de64afa
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/resources/dist.all.first.cleaned
@@ -0,0 +1,5494 @@
+Mary 
+Patricia 
+Linda 
+Barbara 
+Elizabeth 
+Jennifer 
+Maria 
+Susan 
+Margaret 
+Dorothy 
+Lisa 
+Nancy 
+Karen 
+Betty 
+Helen 
+Sandra 
+Donna 
+Carol 
+Ruth 
+Sharon 
+Michelle 
+Laura 
+Sarah 
+Kimberly 
+Deborah 
+Jessica 
+Shirley 
+Cynthia 
+Angela 
+Melissa 
+Brenda 
+Amy 
+Anna 
+Rebecca 
+Virginia 
+Kathleen 
+Pamela 
+Martha 
+Debra 
+Amanda 
+Stephanie 
+Carolyn 
+Christine 
+Marie 
+Janet 
+Catherine 
+Frances 
+Ann 
+Joyce 
+Diane 
+Alice 
+Julie 
+Heather 
+Teresa 
+Doris 
+Gloria 
+Evelyn 
+Jean 
+Cheryl 
+Mildred 
+Katherine 
+Joan 
+Ashley 
+Judith 
+Rose 
+Janice 
+Kelly 
+Nicole 
+Judy 
+Christina 
+Kathy 
+Theresa 
+Beverly 
+Denise 
+Tammy 
+Irene 
+Jane 
+Lori 
+Rachel 
+Marilyn 
+Andrea 
+Kathryn 
+Louise 
+Sara 
+Anne 
+Jacqueline 
+Wanda 
+Bonnie 
+Julia 
+Ruby 
+Lois 
+Tina 
+Phyllis 
+Norma 
+Paula 
+Diana 
+Annie 
+Lillian 
+Emily 
+Robin 
+Peggy 
+Crystal 
+Gladys 
+Rita 
+Dawn 
+Connie 
+Florence 
+Tracy 
+Edna 
+Tiffany 
+Carmen 
+Rosa 
+Cindy 
+Grace 
+Wendy 
+Victoria 
+Edith 
+Kim 
+Sherry 
+Sylvia 
+Josephine 
+Thelma 
+Shannon 
+Sheila 
+Ethel 
+Ellen 
+Elaine 
+Marjorie 
+Carrie 
+Charlotte 
+Monica 
+Esther 
+Pauline 
+Emma 
+Juanita 
+Anita 
+Rhonda 
+Hazel 
+Amber 
+Eva 
+Debbie 
+April 
+Leslie 
+Clara 
+Lucille 
+Jamie 
+Joanne 
+Eleanor 
+Valerie 
+Danielle 
+Megan 
+Alicia 
+Suzanne 
+Michele 
+Gail 
+Bertha 
+Darlene 
+Veronica 
+Jill 
+Erin 
+Geraldine 
+Lauren 
+Cathy 
+Joann 
+Lorraine 
+Lynn 
+Sally 
+Regina 
+Erica 
+Beatrice 
+Dolores 
+Bernice 
+Audrey 
+Yvonne 
+Annette 
+June 
+Samantha 
+Marion 
+Dana 
+Stacy 
+Ana 
+Renee 
+Ida 
+Vivian 
+Roberta 
+Holly 
+Brittany 
+Melanie 
+Loretta 
+Yolanda 
+Jeanette 
+Laurie 
+Katie 
+Kristen 
+Vanessa 
+Alma 
+Sue 
+Elsie 
+Beth 
+Jeanne 
+Vicki 
+Carla 
+Tara 
+Rosemary 
+Eileen 
+Terri 
+Gertrude 
+Lucy 
+Tonya 
+Ella 
+Stacey 
+Wilma 
+Gina 
+Kristin 
+Jessie 
+Natalie 
+Agnes 
+Vera 
+Willie 
+Charlene 
+Bessie 
+Delores 
+Melinda 
+Pearl 
+Arlene 
+Maureen 
+Colleen 
+Allison 
+Tamara 
+Joy 
+Georgia 
+Constance 
+Lillie 
+Claudia 
+Jackie 
+Marcia 
+Tanya 
+Nellie 
+Minnie 
+Marlene 
+Heidi 
+Glenda 
+Lydia 
+Viola 
+Courtney 
+Marian 
+Stella 
+Caroline 
+Dora 
+Jo 
+Vickie 
+Mattie 
+Terry 
+Maxine 
+Irma 
+Mabel 
+Marsha 
+Myrtle 
+Lena 
+Christy 
+Deanna 
+Patsy 
+Hilda 
+Gwendolyn 
+Jennie 
+Nora 
+Margie 
+Nina 
+Cassandra 
+Leah 
+Penny 
+Kay 
+Priscilla 
+Naomi 
+Carole 
+Brandy 
+Olga 
+Billie 
+Dianne 
+Tracey 
+Leona 
+Jenny 
+Felicia 
+Sonia 
+Miriam 
+Velma 
+Becky 
+Bobbie 
+Violet 
+Kristina 
+Toni 
+Misty 
+Mae 
+Shelly 
+Daisy 
+Ramona 
+Sherri 
+Erika 
+Katrina 
+Claire 
+Lindsey 
+Lindsay 
+Geneva 
+Guadalupe 
+Belinda 
+Margarita 
+Sheryl 
+Cora 
+Faye 
+Ada 
+Natasha 
+Sabrina 
+Isabel 
+Marguerite 
+Hattie 
+Harriet 
+Molly 
+Cecilia 
+Kristi 
+Brandi 
+Blanche 
+Sandy 
+Rosie 
+Joanna 
+Iris 
+Eunice 
+Angie 
+Inez 
+Lynda 
+Madeline 
+Amelia 
+Alberta 
+Genevieve 
+Monique 
+Jodi 
+Janie 
+Maggie 
+Kayla 
+Sonya 
+Jan 
+Lee 
+Kristine 
+Candace 
+Fannie 
+Maryann 
+Opal 
+Alison 
+Yvette 
+Melody 
+Luz 
+Susie 
+Olivia 
+Flora 
+Shelley 
+Kristy 
+Mamie 
+Lula 
+Lola 
+Verna 
+Beulah 
+Antoinette 
+Candice 
+Juana 
+Jeannette 
+Pam 
+Kelli 
+Hannah 
+Whitney 
+Bridget 
+Karla 
+Celia 
+Latoya 
+Patty 
+Shelia 
+Gayle 
+Della 
+Vicky 
+Lynne 
+Sheri 
+Marianne 
+Kara 
+Jacquelyn 
+Erma 
+Blanca 
+Myra 
+Leticia 
+Pat 
+Krista 
+Roxanne 
+Angelica 
+Johnnie 
+Robyn 
+Francis 
+Adrienne 
+Rosalie 
+Alexandra 
+Brooke 
+Bethany 
+Sadie 
+Bernadette 
+Traci 
+Jody 
+Kendra 
+Jasmine 
+Nichole 
+Rachael 
+Chelsea 
+Mable 
+Ernestine 
+Muriel 
+Marcella 
+Elena 
+Krystal 
+Angelina 
+Nadine 
+Kari 
+Estelle 
+Dianna 
+Paulette 
+Lora 
+Mona 
+Doreen 
+Rosemarie 
+Angel 
+Desiree 
+Antonia 
+Hope 
+Ginger 
+Janis 
+Betsy 
+Christie 
+Freda 
+Mercedes 
+Meredith 
+Lynette 
+Teri 
+Cristina 
+Eula 
+Leigh 
+Meghan 
+Sophia 
+Eloise 
+Rochelle 
+Gretchen 
+Cecelia 
+Raquel 
+Henrietta 
+Alyssa 
+Jana 
+Kelley 
+Gwen 
+Kerry 
+Jenna 
+Tricia 
+Laverne 
+Olive 
+Alexis 
+Tasha 
+Silvia 
+Elvira 
+Casey 
+Delia 
+Sophie 
+Kate 
+Patti 
+Lorena 
+Kellie 
+Sonja 
+Lila 
+Lana 
+Darla 
+May 
+Mindy 
+Essie 
+Mandy 
+Lorene 
+Elsa 
+Josefina 
+Jeannie 
+Miranda 
+Dixie 
+Lucia 
+Marta 
+Faith 
+Lela 
+Johanna 
+Shari 
+Camille 
+Tami 
+Shawna 
+Elisa 
+Ebony 
+Melba 
+Ora 
+Nettie 
+Tabitha 
+Ollie 
+Jaime 
+Winifred 
+Kristie 
+Marina 
+Alisha 
+Aimee 
+Rena 
+Myrna 
+Marla 
+Tammie 
+Latasha 
+Bonita 
+Patrice 
+Ronda 
+Sherrie 
+Addie 
+Francine 
+Deloris 
+Stacie 
+Adriana 
+Cheri 
+Shelby 
+Abigail 
+Celeste 
+Jewel 
+Cara 
+Adele 
+Rebekah 
+Lucinda 
+Dorthy 
+Chris 
+Effie 
+Trina 
+Reba 
+Shawn 
+Sallie 
+Aurora 
+Lenora 
+Etta 
+Lottie 
+Kerri 
+Trisha 
+Nikki 
+Estella 
+Francisca 
+Josie 
+Tracie 
+Marissa 
+Karin 
+Brittney 
+Janelle 
+Lourdes 
+Laurel 
+Helene 
+Fern 
+Elva 
+Corinne 
+Kelsey 
+Ina 
+Bettie 
+Elisabeth 
+Aida 
+Caitlin 
+Ingrid 
+Iva 
+Eugenia 
+Christa 
+Goldie 
+Cassie 
+Maude 
+Jenifer 
+Therese 
+Frankie 
+Dena 
+Lorna 
+Janette 
+Latonya 
+Candy 
+Morgan 
+Consuelo 
+Tamika 
+Rosetta 
+Debora 
+Cherie 
+Polly 
+Dina 
+Jewell 
+Fay 
+Jillian 
+Dorothea 
+Nell 
+Trudy 
+Esperanza 
+Patrica 
+Kimberley 
+Shanna 
+Helena 
+Carolina 
+Cleo 
+Stefanie 
+Rosario 
+Ola 
+Janine 
+Mollie 
+Lupe 
+Alisa 
+Lou 
+Maribel 
+Susanne 
+Bette 
+Susana 
+Elise 
+Cecile 
+Isabelle 
+Lesley 
+Jocelyn 
+Paige 
+Joni 
+Rachelle 
+Leola 
+Daphne 
+Alta 
+Ester 
+Petra 
+Graciela 
+Imogene 
+Jolene 
+Keisha 
+Lacey 
+Glenna 
+Gabriela 
+Keri 
+Ursula 
+Lizzie 
+Kirsten 
+Shana 
+Adeline 
+Mayra 
+Jayne 
+Jaclyn 
+Gracie 
+Sondra 
+Carmela 
+Marisa 
+Rosalind 
+Charity 
+Tonia 
+Beatriz 
+Marisol 
+Clarice 
+Jeanine 
+Sheena 
+Angeline 
+Frieda 
+Lily 
+Robbie 
+Shauna 
+Millie 
+Claudette 
+Cathleen 
+Angelia 
+Gabrielle 
+Autumn 
+Katharine 
+Summer 
+Jodie 
+Staci 
+Lea 
+Christi 
+Jimmie 
+Justine 
+Elma 
+Luella 
+Margret 
+Dominique 
+Socorro 
+Rene 
+Martina 
+Margo 
+Mavis 
+Callie 
+Bobbi 
+Maritza 
+Lucile 
+Leanne 
+Jeannine 
+Deana 
+Aileen 
+Lorie 
+Ladonna 
+Willa 
+Manuela 
+Gale 
+Selma 
+Dolly 
+Sybil 
+Abby 
+Lara 
+Dale 
+Ivy 
+Dee 
+Winnie 
+Marcy 
+Luisa 
+Jeri 
+Magdalena 
+Ofelia 
+Meagan 
+Audra 
+Matilda 
+Leila 
+Cornelia 
+Bianca 
+Simone 
+Bettye 
+Randi 
+Virgie 
+Latisha 
+Barbra 
+Georgina 
+Eliza 
+Leann 
+Bridgette 
+Rhoda 
+Haley 
+Adela 
+Nola 
+Bernadine 
+Flossie 
+Ila 
+Greta 
+Ruthie 
+Nelda 
+Minerva 
+Lilly 
+Terrie 
+Letha 
+Hilary 
+Estela 
+Valarie 
+Brianna 
+Rosalyn 
+Earline 
+Catalina 
+Ava 
+Mia 
+Clarissa 
+Lidia 
+Corrine 
+Alexandria 
+Concepcion 
+Tia 
+Sharron 
+Rae 
+Dona 
+Ericka 
+Jami 
+Elnora 
+Chandra 
+Lenore 
+Neva 
+Marylou 
+Melisa 
+Tabatha 
+Serena 
+Avis 
+Allie 
+Sofia 
+Jeanie 
+Odessa 
+Nannie 
+Harriett 
+Loraine 
+Penelope 
+Milagros 
+Emilia 
+Benita 
+Allyson 
+Ashlee 
+Tania 
+Tommie 
+Esmeralda 
+Karina 
+Eve 
+Pearlie 
+Zelma 
+Malinda 
+Noreen 
+Tameka 
+Saundra 
+Hillary 
+Amie 
+Althea 
+Rosalinda 
+Jordan 
+Lilia 
+Alana 
+Gay 
+Clare 
+Alejandra 
+Elinor 
+Michael 
+Lorrie 
+Jerri 
+Darcy 
+Earnestine 
+Carmella 
+Taylor 
+Noemi 
+Marcie 
+Liza 
+Annabelle 
+Louisa 
+Earlene 
+Mallory 
+Carlene 
+Nita 
+Selena 
+Tanisha 
+Katy 
+Julianne 
+John 
+Lakisha 
+Edwina 
+Maricela 
+Margery 
+Kenya 
+Dollie 
+Roxie 
+Roslyn 
+Kathrine 
+Nanette 
+Charmaine 
+Lavonne 
+Ilene 
+Kris 
+Tammi 
+Suzette 
+Corine 
+Kaye 
+Jerry 
+Merle 
+Chrystal 
+Lina 
+Deanne 
+Lilian 
+Juliana 
+Aline 
+Luann 
+Kasey 
+Maryanne 
+Evangeline 
+Colette 
+Melva 
+Lawanda 
+Yesenia 
+Nadia 
+Madge 
+Kathie 
+Eddie 
+Ophelia 
+Valeria 
+Nona 
+Mitzi 
+Mari 
+Georgette 
+Claudine 
+Fran 
+Alissa 
+Roseann 
+Lakeisha 
+Susanna 
+Reva 
+Deidre 
+Chasity 
+Sheree 
+Carly 
+James 
+Elvia 
+Alyce 
+Deirdre 
+Gena 
+Briana 
+Araceli 
+Katelyn 
+Rosanne 
+Wendi 
+Tessa 
+Berta 
+Marva 
+Imelda 
+Marietta 
+Marci 
+Leonor 
+Arline 
+Sasha 
+Madelyn 
+Janna 
+Juliette 
+Deena 
+Aurelia 
+Josefa 
+Augusta 
+Liliana 
+Young 
+Christian 
+Lessie 
+Amalia 
+Savannah 
+Anastasia 
+Vilma 
+Natalia 
+Rosella 
+Lynnette 
+Corina 
+Alfreda 
+Leanna 
+Carey 
+Amparo 
+Coleen 
+Tamra 
+Aisha 
+Wilda 
+Karyn 
+Cherry 
+Queen 
+Maura 
+Mai 
+Evangelina 
+Rosanna 
+Hallie 
+Erna 
+Enid 
+Mariana 
+Lacy 
+Juliet 
+Jacklyn 
+Freida 
+Madeleine 
+Mara 
+Hester 
+Cathryn 
+Lelia 
+Casandra 
+Bridgett 
+Angelita 
+Jannie 
+Dionne 
+Annmarie 
+Katina 
+Beryl 
+Phoebe 
+Millicent 
+Katheryn 
+Diann 
+Carissa 
+Maryellen 
+Liz 
+Lauri 
+Helga 
+Gilda 
+Adrian 
+Rhea 
+Marquita 
+Hollie 
+Tisha 
+Tamera 
+Angelique 
+Francesca 
+Britney 
+Kaitlin 
+Lolita 
+Florine 
+Rowena 
+Reyna 
+Twila 
+Fanny 
+Janell 
+Ines 
+Concetta 
+Bertie 
+Alba 
+Brigitte 
+Alyson 
+Vonda 
+Pansy 
+Elba 
+Noelle 
+Letitia 
+Kitty 
+Deann 
+Brandie 
+Louella 
+Leta 
+Felecia 
+Sharlene 
+Lesa 
+Beverley 
+Robert 
+Isabella 
+Herminia 
+Terra 
+Celina 
+Tori 
+Octavia 
+Jade 
+Denice 
+Germaine 
+Sierra 
+Michell 
+Cortney 
+Nelly 
+Doretha 
+Sydney 
+Deidra 
+Monika 
+Lashonda 
+Judi 
+Chelsey 
+Antionette 
+Margot 
+Bobby 
+Adelaide 
+Nan 
+Leeann 
+Elisha 
+Dessie 
+Libby 
+Kathi 
+Gayla 
+Latanya 
+Mina 
+Mellisa 
+Kimberlee 
+Jasmin 
+Renae 
+Zelda 
+Elda 
+Ma 
+Justina 
+Gussie 
+Emilie 
+Camilla 
+Abbie 
+Rocio 
+Kaitlyn 
+Jesse 
+Edythe 
+Ashleigh 
+Selina 
+Lakesha 
+Geri 
+Allene 
+Pamala 
+Michaela 
+Dayna 
+Caryn 
+Rosalia 
+Sun 
+Jacquline 
+Rebeca 
+Marybeth 
+Krystle 
+Iola 
+Dottie 
+Bennie 
+Belle 
+Aubrey 
+Griselda 
+Ernestina 
+Elida 
+Adrianne 
+Demetria 
+Delma 
+Chong 
+Jaqueline 
+Destiny 
+Arleen 
+Virgina 
+Retha 
+Fatima 
+Tillie 
+Eleanore 
+Cari 
+Treva 
+Birdie 
+Wilhelmina 
+Rosalee 
+Maurine 
+Latrice 
+Yong 
+Jena 
+Taryn 
+Elia 
+Debby 
+Maudie 
+Jeanna 
+Delilah 
+Catrina 
+Shonda 
+Hortencia 
+Theodora 
+Teresita 
+Robbin 
+Danette 
+Maryjane 
+Freddie 
+Delphine 
+Brianne 
+Nilda 
+Danna 
+Cindi 
+Bess 
+Iona 
+Hanna 
+Ariel 
+Winona 
+Vida 
+Rosita 
+Marianna 
+William 
+Racheal 
+Guillermina 
+Eloisa 
+Celestine 
+Caren 
+Malissa 
+Lona 
+Chantel 
+Shellie 
+Marisela 
+Leora 
+Agatha 
+Soledad 
+Migdalia 
+Ivette 
+Christen 
+Athena 
+Janel 
+Chloe 
+Veda 
+Pattie 
+Tessie 
+Tera 
+Marilynn 
+Lucretia 
+Karrie 
+Dinah 
+Daniela 
+Alecia 
+Adelina 
+Vernice 
+Shiela 
+Portia 
+Merry 
+Lashawn 
+Devon 
+Dara 
+Tawana 
+Oma 
+Verda 
+Christin 
+Alene 
+Zella 
+Sandi 
+Rafaela 
+Maya 
+Kira 
+Candida 
+Alvina 
+Suzan 
+Shayla 
+Lyn 
+Lettie 
+Alva 
+Samatha 
+Oralia 
+Matilde 
+Madonna 
+Larissa 
+Vesta 
+Renita 
+India 
+Delois 
+Shanda 
+Phillis 
+Lorri 
+Erlinda 
+Cruz 
+Cathrine 
+Barb 
+Zoe 
+Isabell 
+Ione 
+Gisela 
+Charlie 
+Valencia 
+Roxanna 
+Mayme 
+Kisha 
+Ellie 
+Mellissa 
+Dorris 
+Dalia 
+Bella 
+Annetta 
+Zoila 
+Reta 
+Reina 
+Lauretta 
+Kylie 
+Christal 
+Pilar 
+Charla 
+Elissa 
+Tiffani 
+Tana 
+Paulina 
+Leota 
+Breanna 
+Jayme 
+Carmel 
+Vernell 
+Tomasa 
+Mandi 
+Dominga 
+Santa 
+Melodie 
+Lura 
+Alexa 
+Tamela 
+Ryan 
+Mirna 
+Kerrie 
+Venus 
+Noel 
+Felicita 
+Cristy 
+Carmelita 
+Berniece 
+Annemarie 
+Tiara 
+Roseanne 
+Missy 
+Cori 
+Roxana 
+Pricilla 
+Kristal 
+Jung 
+Elyse 
+Haydee 
+Aletha 
+Bettina 
+Marge 
+Gillian 
+Filomena 
+Charles 
+Zenaida 
+Harriette 
+Caridad 
+Vada 
+Una 
+Aretha 
+Pearline 
+Marjory 
+Marcela 
+Flor 
+Evette 
+Elouise 
+Alina 
+Trinidad 
+David 
+Damaris 
+Catharine 
+Carroll 
+Belva 
+Nakia 
+Marlena 
+Luanne 
+Lorine 
+Karon 
+Dorene 
+Danita 
+Brenna 
+Tatiana 
+Sammie 
+Louann 
+Loren 
+Julianna 
+Andria 
+Philomena 
+Lucila 
+Leonora 
+Dovie 
+Romona 
+Mimi 
+Jacquelin 
+Gaye 
+Tonja 
+Misti 
+Joe 
+Gene 
+Chastity 
+Stacia 
+Roxann 
+Micaela 
+Nikita 
+Mei 
+Velda 
+Marlys 
+Johnna 
+Aura 
+Lavern 
+Ivonne 
+Hayley 
+Nicki 
+Majorie 
+Herlinda 
+George 
+Alpha 
+Yadira 
+Perla 
+Gregoria 
+Daniel 
+Antonette 
+Shelli 
+Mozelle 
+Mariah 
+Joelle 
+Cordelia 
+Josette 
+Chiquita 
+Trista 
+Louis 
+Laquita 
+Georgiana 
+Candi 
+Shanon 
+Lonnie 
+Hildegard 
+Cecil 
+Valentina 
+Stephany 
+Magda 
+Karol 
+Gerry 
+Gabriella 
+Tiana 
+Roma 
+Richelle 
+Ray 
+Princess 
+Oleta 
+Jacque 
+Idella 
+Alaina 
+Suzanna 
+Jovita 
+Blair 
+Tosha 
+Raven 
+Nereida 
+Marlyn 
+Kyla 
+Joseph 
+Delfina 
+Tena 
+Stephenie 
+Sabina 
+Nathalie 
+Marcelle 
+Gertie 
+Darleen 
+Thea 
+Sharonda 
+Shantel 
+Belen 
+Venessa 
+Rosalina 
+Ona 
+Genoveva 
+Corey 
+Clementine 
+Rosalba 
+Renate 
+Renata 
+Mi 
+Ivory 
+Georgianna 
+Floy 
+Dorcas 
+Ariana 
+Tyra 
+Theda 
+Mariam 
+Juli 
+Jesica 
+Donnie 
+Vikki 
+Verla 
+Roselyn 
+Melvina 
+Jannette 
+Ginny 
+Debrah 
+Corrie 
+Asia 
+Violeta 
+Myrtis 
+Latricia 
+Collette 
+Charleen 
+Anissa 
+Viviana 
+Twyla 
+Precious 
+Nedra 
+Latonia 
+Lan 
+Hellen 
+Fabiola 
+Annamarie 
+Adell 
+Sharyn 
+Chantal 
+Niki 
+Maud 
+Lizette 
+Lindy 
+Kia 
+Kesha 
+Jeana 
+Danelle 
+Charline 
+Chanel 
+Carrol 
+Valorie 
+Lia 
+Dortha 
+Cristal 
+Sunny 
+Leone 
+Leilani 
+Gerri 
+Debi 
+Andra 
+Keshia 
+Ima 
+Eulalia 
+Easter 
+Dulce 
+Natividad 
+Linnie 
+Kami 
+Georgie 
+Catina 
+Brook 
+Alda 
+Winnifred 
+Sharla 
+Ruthann 
+Meaghan 
+Magdalene 
+Lissette 
+Adelaida 
+Venita 
+Trena 
+Shirlene 
+Shameka 
+Elizebeth 
+Dian 
+Shanta 
+Mickey 
+Latosha 
+Carlotta 
+Windy 
+Soon 
+Rosina 
+Mariann 
+Leisa 
+Jonnie 
+Dawna 
+Cathie 
+Billy 
+Astrid 
+Sidney 
+Laureen 
+Janeen 
+Holli 
+Fawn 
+Vickey 
+Teressa 
+Shante 
+Rubye 
+Marcelina 
+Chanda 
+Cary 
+Terese 
+Scarlett 
+Marty 
+Marnie 
+Lulu 
+Lisette 
+Jeniffer 
+Elenor 
+Dorinda 
+Donita 
+Carman 
+Bernita 
+Altagracia 
+Aleta 
+Adrianna 
+Zoraida 
+Ronnie 
+Nicola 
+Lyndsey 
+Kendall 
+Janina 
+Chrissy 
+Ami 
+Starla 
+Phylis 
+Phuong 
+Kyra 
+Charisse 
+Blanch 
+Sanjuanita 
+Rona 
+Nanci 
+Marilee 
+Maranda 
+Cory 
+Brigette 
+Sanjuana 
+Marita 
+Kassandra 
+Joycelyn 
+Ira 
+Felipa 
+Chelsie 
+Bonny 
+Mireya 
+Lorenza 
+Kyong 
+Ileana 
+Candelaria 
+Tony 
+Toby 
+Sherie 
+Ok 
+Mark 
+Lucie 
+Leatrice 
+Lakeshia 
+Gerda 
+Edie 
+Bambi 
+Marylin 
+Lavon 
+Hortense 
+Garnet 
+Evie 
+Tressa 
+Shayna 
+Lavina 
+Kyung 
+Jeanetta 
+Sherrill 
+Shara 
+Phyliss 
+Mittie 
+Anabel 
+Alesia 
+Thuy 
+Tawanda 
+Richard 
+Joanie 
+Tiffanie 
+Lashanda 
+Karissa 
+Enriqueta 
+Daria 
+Daniella 
+Corinna 
+Alanna 
+Abbey 
+Roxane 
+Roseanna 
+Magnolia 
+Lida 
+Kyle 
+Joellen 
+Era 
+Coral 
+Carleen 
+Tresa 
+Peggie 
+Novella 
+Nila 
+Maybelle 
+Jenelle 
+Carina 
+Nova 
+Melina 
+Marquerite 
+Margarette 
+Josephina 
+Evonne 
+Devin 
+Cinthia 
+Albina 
+Toya 
+Tawnya 
+Sherita 
+Santos 
+Myriam 
+Lizabeth 
+Lise 
+Keely 
+Jenni 
+Giselle 
+Cheryle 
+Ardith 
+Ardis 
+Alesha 
+Adriane 
+Shaina 
+Linnea 
+Karolyn 
+Hong 
+Florida 
+Felisha 
+Dori 
+Darci 
+Artie 
+Armida 
+Zola 
+Xiomara 
+Vergie 
+Shamika 
+Nena 
+Nannette 
+Maxie 
+Lovie 
+Jeane 
+Jaimie 
+Inge 
+Farrah 
+Elaina 
+Caitlyn 
+Starr 
+Felicitas 
+Cherly 
+Caryl 
+Yolonda 
+Yasmin 
+Teena 
+Prudence 
+Pennie 
+Nydia 
+Mackenzie 
+Orpha 
+Marvel 
+Lizbeth 
+Laurette 
+Jerrie 
+Hermelinda 
+Carolee 
+Tierra 
+Mirian 
+Meta 
+Melony 
+Kori 
+Jennette 
+Jamila 
+Ena 
+Anh 
+Yoshiko 
+Susannah 
+Salina 
+Rhiannon 
+Joleen 
+Cristine 
+Ashton 
+Aracely 
+Tomeka 
+Shalonda 
+Marti 
+Lacie 
+Kala 
+Jada 
+Ilse 
+Hailey 
+Brittani 
+Zona 
+Syble 
+Sherryl 
+Randy 
+Nidia 
+Marlo 
+Kandice 
+Kandi 
+Deb 
+Dean 
+America 
+Alycia 
+Tommy 
+Ronna 
+Norene 
+Mercy 
+Jose 
+Ingeborg 
+Giovanna 
+Gemma 
+Christel 
+Audry 
+Zora 
+Vita 
+Van 
+Trish 
+Stephaine 
+Shirlee 
+Shanika 
+Melonie 
+Mazie 
+Jazmin 
+Inga 
+Hoa 
+Hettie 
+Geralyn 
+Fonda 
+Estrella 
+Adella 
+Su 
+Sarita 
+Rina 
+Milissa 
+Maribeth 
+Golda 
+Evon 
+Ethelyn 
+Enedina 
+Cherise 
+Chana 
+Velva 
+Tawanna 
+Sade 
+Mirta 
+Li 
+Karie 
+Jacinta 
+Elna 
+Davina 
+Cierra 
+Ashlie 
+Albertha 
+Tanesha 
+Stephani 
+Nelle 
+Mindi 
+Lu 
+Lorinda 
+Larue 
+Florene 
+Demetra 
+Dedra 
+Ciara 
+Chantelle 
+Ashly 
+Suzy 
+Rosalva 
+Noelia 
+Lyda 
+Leatha 
+Krystyna 
+Kristan 
+Karri 
+Darline 
+Darcie 
+Cinda 
+Cheyenne 
+Cherrie 
+Awilda 
+Almeda 
+Rolanda 
+Lanette 
+Jerilyn 
+Gisele 
+Evalyn 
+Cyndi 
+Cleta 
+Carin 
+Zina 
+Zena 
+Velia 
+Tanika 
+Paul 
+Charissa 
+Thomas 
+Talia 
+Margarete 
+Lavonda 
+Kaylee 
+Kathlene 
+Jonna 
+Irena 
+Ilona 
+Idalia 
+Candis 
+Candance 
+Brandee 
+Anitra 
+Alida 
+Sigrid 
+Nicolette 
+Maryjo 
+Linette 
+Hedwig 
+Christiana 
+Cassidy 
+Alexia 
+Tressie 
+Modesta 
+Lupita 
+Lita 
+Gladis 
+Evelia 
+Davida 
+Cherri 
+Cecily 
+Ashely 
+Annabel 
+Agustina 
+Wanita 
+Shirly 
+Rosaura 
+Hulda 
+Eun 
+Bailey 
+Yetta 
+Verona 
+Thomasina 
+Sibyl 
+Shannan 
+Mechelle 
+Lue 
+Leandra 
+Lani 
+Kylee 
+Kandy 
+Jolynn 
+Ferne 
+Eboni 
+Corene 
+Alysia 
+Zula 
+Nada 
+Moira 
+Lyndsay 
+Lorretta 
+Juan 
+Jammie 
+Hortensia 
+Gaynell 
+Cameron 
+Adria 
+Vina 
+Vicenta 
+Tangela 
+Stephine 
+Norine 
+Nella 
+Liana 
+Leslee 
+Kimberely 
+Iliana 
+Glory 
+Felica 
+Emogene 
+Elfriede 
+Eden 
+Eartha 
+Carma 
+Bea 
+Ocie 
+Marry 
+Lennie 
+Kiara 
+Jacalyn 
+Carlota 
+Arielle 
+Yu 
+Star 
+Otilia 
+Kirstin 
+Kacey 
+Johnetta 
+Joey 
+Joetta 
+Jeraldine 
+Jaunita 
+Elana 
+Dorthea 
+Cami 
+Amada 
+Adelia 
+Vernita 
+Tamar 
+Siobhan 
+Renea 
+Rashida 
+Ouida 
+Odell 
+Nilsa 
+Meryl 
+Kristyn 
+Julieta 
+Danica 
+Breanne 
+Aurea 
+Anglea 
+Sherron 
+Odette 
+Malia 
+Lorelei 
+Lin 
+Leesa 
+Kenna 
+Kathlyn 
+Fiona 
+Charlette 
+Suzie 
+Shantell 
+Sabra 
+Racquel 
+Myong 
+Mira 
+Martine 
+Lucienne 
+Lavada 
+Juliann 
+Johnie 
+Elvera 
+Delphia 
+Clair 
+Christiane 
+Charolette 
+Carri 
+Augustine 
+Asha 
+Angella 
+Paola 
+Ninfa 
+Leda 
+Lai 
+Eda 
+Sunshine 
+Stefani 
+Shanell 
+Palma 
+Machelle 
+Lissa 
+Kecia 
+Kathryne 
+Karlene 
+Julissa 
+Jettie 
+Jenniffer 
+Hui 
+Corrina 
+Christopher 
+Carolann 
+Alena 
+Tess 
+Rosaria 
+Myrtice 
+Marylee 
+Liane 
+Kenyatta 
+Judie 
+Janey 
+In 
+Elmira 
+Eldora 
+Denna 
+Cristi 
+Cathi 
+Zaida 
+Vonnie 
+Viva 
+Vernie 
+Rosaline 
+Mariela 
+Luciana 
+Lesli 
+Karan 
+Felice 
+Deneen 
+Adina 
+Wynona 
+Tarsha 
+Sheron 
+Shasta 
+Shanita 
+Shani 
+Shandra 
+Randa 
+Pinkie 
+Paris 
+Nelida 
+Marilou 
+Lyla 
+Laurene 
+Laci 
+Joi 
+Janene 
+Dorotha 
+Daniele 
+Dani 
+Carolynn 
+Carlyn 
+Berenice 
+Ayesha 
+Anneliese 
+Alethea 
+Thersa 
+Tamiko 
+Rufina 
+Oliva 
+Mozell 
+Marylyn 
+Madison 
+Kristian 
+Kathyrn 
+Kasandra 
+Kandace 
+Janae 
+Gabriel 
+Domenica 
+Debbra 
+Dannielle 
+Chun 
+Buffy 
+Barbie 
+Arcelia 
+Aja 
+Zenobia 
+Sharen 
+Sharee 
+Patrick 
+Page 
+My 
+Lavinia 
+Kum 
+Kacie 
+Jackeline 
+Huong 
+Felisa 
+Emelia 
+Eleanora 
+Cythia 
+Cristin 
+Clyde 
+Claribel 
+Caron 
+Anastacia 
+Zulma 
+Zandra 
+Yoko 
+Tenisha 
+Susann 
+Sherilyn 
+Shay 
+Shawanda 
+Sabine 
+Romana 
+Mathilda 
+Linsey 
+Keiko 
+Joana 
+Isela 
+Gretta 
+Georgetta 
+Eugenie 
+Dusty 
+Desirae 
+Delora 
+Corazon 
+Antonina 
+Anika 
+Willene 
+Tracee 
+Tamatha 
+Regan 
+Nichelle 
+Mickie 
+Maegan 
+Luana 
+Lanita 
+Kelsie 
+Edelmira 
+Bree 
+Afton 
+Teodora 
+Tamie 
+Shena 
+Meg 
+Linh 
+Keli 
+Kaci 
+Danyelle 
+Britt 
+Arlette 
+Albertine 
+Adelle 
+Tiffiny 
+Stormy 
+Simona 
+Numbers 
+Nicolasa 
+Nichol 
+Nia 
+Nakisha 
+Mee 
+Maira 
+Loreen 
+Kizzy 
+Johnny 
+Jay 
+Fallon 
+Christene 
+Bobbye 
+Anthony 
+Ying 
+Vincenza 
+Tanja 
+Rubie 
+Roni 
+Queenie 
+Margarett 
+Kimberli 
+Irmgard 
+Idell 
+Hilma 
+Evelina 
+Esta 
+Emilee 
+Dennise 
+Dania 
+Carl 
+Carie 
+Antonio 
+Wai 
+Sang 
+Risa 
+Rikki 
+Particia 
+Mui 
+Masako 
+Mario 
+Luvenia 
+Loree 
+Loni 
+Lien 
+Kevin 
+Gigi 
+Florencia 
+Dorian 
+Denita 
+Dallas 
+Chi 
+Billye 
+Alexander 
+Tomika 
+Sharita 
+Rana 
+Nikole 
+Neoma 
+Margarite 
+Madalyn 
+Lucina 
+Laila 
+Kali 
+Jenette 
+Gabriele 
+Evelyne 
+Elenora 
+Clementina 
+Alejandrina 
+Zulema 
+Violette 
+Vannessa 
+Thresa 
+Retta 
+Pia 
+Patience 
+Noella 
+Nickie 
+Jonell 
+Delta 
+Chung 
+Chaya 
+Camelia 
+Bethel 
+Anya 
+Andrew 
+Thanh 
+Suzann 
+Spring 
+Shu 
+Mila 
+Lilla 
+Laverna 
+Keesha 
+Kattie 
+Gia 
+Georgene 
+Eveline 
+Estell 
+Elizbeth 
+Vivienne 
+Vallie 
+Trudie 
+Stephane 
+Michel 
+Magaly 
+Madie 
+Kenyetta 
+Karren 
+Janetta 
+Hermine 
+Harmony 
+Drucilla 
+Debbi 
+Celestina 
+Candie 
+Britni 
+Beckie 
+Amina 
+Zita 
+Yun 
+Yolande 
+Vivien 
+Vernetta 
+Trudi 
+Sommer 
+Pearle 
+Patrina 
+Ossie 
+Nicolle 
+Loyce 
+Letty 
+Larisa 
+Katharina 
+Joselyn 
+Jonelle 
+Jenell 
+Iesha 
+Heide 
+Florinda 
+Florentina 
+Flo 
+Elodia 
+Dorine 
+Brunilda 
+Brigid 
+Ashli 
+Ardella 
+Twana 
+Thu 
+Tarah 
+Sung 
+Shea 
+Shavon 
+Shane 
+Serina 
+Rayna 
+Ramonita 
+Nga 
+Margurite 
+Lucrecia 
+Kourtney 
+Kati 
+Jesus 
+Jesenia 
+Diamond 
+Crista 
+Ayana 
+Alica 
+Alia 
+Vinnie 
+Suellen 
+Romelia 
+Rachell 
+Piper 
+Olympia 
+Michiko 
+Kathaleen 
+Jolie 
+Jessi 
+Janessa 
+Hana 
+Ha 
+Elease 
+Carletta 
+Britany 
+Shona 
+Salome 
+Rosamond 
+Regena 
+Raina 
+Ngoc 
+Nelia 
+Louvenia 
+Lesia 
+Latrina 
+Laticia 
+Larhonda 
+Jina 
+Jacki 
+Hollis 
+Holley 
+Emmy 
+Deeann 
+Coretta 
+Arnetta 
+Velvet 
+Thalia 
+Shanice 
+Neta 
+Mikki 
+Micki 
+Lonna 
+Leana 
+Lashunda 
+Kiley 
+Joye 
+Jacqulyn 
+Ignacia 
+Hyun 
+Hiroko 
+Henry 
+Henriette 
+Elayne 
+Delinda 
+Darnell 
+Dahlia 
+Coreen 
+Consuela 
+Conchita 
+Celine 
+Babette 
+Ayanna 
+Anette 
+Albertina 
+Skye 
+Shawnee 
+Shaneka 
+Quiana 
+Pamelia 
+Min 
+Merri 
+Merlene 
+Margit 
+Kiesha 
+Kiera 
+Kaylene 
+Jodee 
+Jenise 
+Erlene 
+Emmie 
+Else 
+Daryl 
+Dalila 
+Daisey 
+Cody 
+Casie 
+Belia 
+Babara 
+Versie 
+Vanesa 
+Shelba 
+Shawnda 
+Sam 
+Norman 
+Nikia 
+Naoma 
+Marna 
+Margeret 
+Madaline 
+Lawana 
+Kindra 
+Jutta 
+Jazmine 
+Janett 
+Hannelore 
+Glendora 
+Gertrud 
+Garnett 
+Freeda 
+Frederica 
+Florance 
+Flavia 
+Dennis 
+Carline 
+Beverlee 
+Anjanette 
+Valda 
+Trinity 
+Tamala 
+Stevie 
+Shonna 
+Sha 
+Sarina 
+Oneida 
+Micah 
+Merilyn 
+Marleen 
+Lurline 
+Lenna 
+Katherin 
+Jin 
+Jeni 
+Hae 
+Gracia 
+Glady 
+Farah 
+Eric 
+Enola 
+Ema 
+Dominque 
+Devona 
+Delana 
+Cecila 
+Caprice 
+Alysha 
+Ali 
+Alethia 
+Vena 
+Theresia 
+Tawny 
+Song 
+Shakira 
+Samara 
+Sachiko 
+Rachele 
+Pamella 
+Nicky 
+Marni 
+Mariel 
+Maren 
+Malisa 
+Ligia 
+Lera 
+Latoria 
+Larae 
+Kimber 
+Kathern 
+Karey 
+Jennefer 
+Janeth 
+Halina 
+Fredia 
+Delisa 
+Debroah 
+Ciera 
+Chin 
+Angelika 
+Andree 
+Altha 
+Yen 
+Vivan 
+Terresa 
+Tanna 
+Suk 
+Sudie 
+Soo 
+Signe 
+Salena 
+Ronni 
+Rebbecca 
+Myrtie 
+Mckenzie 
+Malika 
+Maida 
+Loan 
+Leonarda 
+Kayleigh 
+France 
+Ethyl 
+Ellyn 
+Dayle 
+Cammie 
+Brittni 
+Birgit 
+Avelina 
+Asuncion 
+Arianna 
+Akiko 
+Venice 
+Tyesha 
+Tonie 
+Tiesha 
+Takisha 
+Steffanie 
+Sindy 
+Santana 
+Meghann 
+Manda 
+Macie 
+Lady 
+Kellye 
+Kellee 
+Joslyn 
+Jason 
+Inger 
+Indira 
+Glinda 
+Glennis 
+Fernanda 
+Faustina 
+Eneida 
+Elicia 
+Dot 
+Digna 
+Dell 
+Arletta 
+Andre 
+Willia 
+Tammara 
+Tabetha 
+Sherrell 
+Sari 
+Refugio 
+Rebbeca 
+Pauletta 
+Nieves 
+Natosha 
+Nakita 
+Mammie 
+Kenisha 
+Kazuko 
+Kassie 
+Gary 
+Earlean 
+Daphine 
+Corliss 
+Clotilde 
+Carolyne 
+Bernetta 
+Augustina 
+Audrea 
+Annis 
+Annabell 
+Yan 
+Tennille 
+Tamica 
+Selene 
+Sean 
+Rosana 
+Regenia 
+Qiana 
+Markita 
+Macy 
+Leeanne 
+Laurine 
+Kym 
+Jessenia 
+Janita 
+Georgine 
+Genie 
+Emiko 
+Elvie 
+Deandra 
+Dagmar 
+Corie 
+Collen 
+Cherish 
+Romaine 
+Porsha 
+Pearlene 
+Micheline 
+Merna 
+Margorie 
+Margaretta 
+Lore 
+Kenneth 
+Jenine 
+Hermina 
+Fredericka 
+Elke 
+Drusilla 
+Dorathy 
+Dione 
+Desire 
+Celena 
+Brigida 
+Angeles 
+Allegra 
+Theo 
+Tamekia 
+Synthia 
+Stephen 
+Sook 
+Slyvia 
+Rosann 
+Reatha 
+Raye 
+Marquetta 
+Margart 
+Ling 
+Layla 
+Kymberly 
+Kiana 
+Kayleen 
+Katlyn 
+Karmen 
+Joella 
+Irina 
+Emelda 
+Eleni 
+Detra 
+Clemmie 
+Cheryll 
+Chantell 
+Cathey 
+Arnita 
+Arla 
+Angle 
+Angelic 
+Alyse 
+Zofia 
+Thomasine 
+Tennie 
+Son 
+Sherly 
+Sherley 
+Sharyl 
+Remedios 
+Petrina 
+Nickole 
+Myung 
+Myrle 
+Mozella 
+Louanne 
+Lisha 
+Latia 
+Lane 
+Krysta 
+Julienne 
+Joel 
+Jeanene 
+Jacqualine 
+Isaura 
+Gwenda 
+Earleen 
+Donald 
+Cleopatra 
+Carlie 
+Audie 
+Antonietta 
+Alise 
+Alex 
+Verdell 
+Val 
+Tyler 
+Tomoko 
+Thao 
+Talisha 
+Steven 
+So 
+Shemika 
+Shaun 
+Scarlet 
+Savanna 
+Santina 
+Rosia 
+Raeann 
+Odilia 
+Nana 
+Minna 
+Magan 
+Lynelle 
+Le 
+Karma 
+Joeann 
+Ivana 
+Inell 
+Ilana 
+Hye 
+Honey 
+Hee 
+Gudrun 
+Frank 
+Dreama 
+Crissy 
+Chante 
+Carmelina 
+Arvilla 
+Arthur 
+Annamae 
+Alvera 
+Aleida 
+Aaron 
+Yee 
+Yanira 
+Vanda 
+Tianna 
+Tam 
+Stefania 
+Shira 
+Perry 
+Nicol 
+Nancie 
+Monserrate 
+Minh 
+Melynda 
+Melany 
+Matthew 
+Lovella 
+Laure 
+Kirby 
+Kacy 
+Jacquelynn 
+Hyon 
+Gertha 
+Francisco 
+Eliana 
+Christena 
+Christeen 
+Charise 
+Caterina 
+Carley 
+Candyce 
+Arlena 
+Ammie 
+Yang 
+Willette 
+Vanita 
+Tuyet 
+Tiny 
+Syreeta 
+Silva 
+Scott 
+Ronald 
+Penney 
+Nyla 
+Michal 
+Maurice 
+Maryam 
+Marya 
+Magen 
+Ludie 
+Loma 
+Livia 
+Lanell 
+Kimberlie 
+Julee 
+Donetta 
+Diedra 
+Denisha 
+Deane 
+Dawne 
+Clarine 
+Cherryl 
+Bronwyn 
+Brandon 
+Alla 
+Valery 
+Tonda 
+Sueann 
+Soraya 
+Shoshana 
+Shela 
+Sharleen 
+Shanelle 
+Nerissa 
+Micheal 
+Meridith 
+Mellie 
+Maye 
+Maple 
+Magaret 
+Luis 
+Lili 
+Leonila 
+Leonie 
+Leeanna 
+Lavonia 
+Lavera 
+Kristel 
+Kathey 
+Kathe 
+Justin 
+Julian 
+Jimmy 
+Jann 
+Ilda 
+Hildred 
+Hildegarde 
+Genia 
+Fumiko 
+Evelin 
+Ermelinda 
+Elly 
+Dung 
+Doloris 
+Dionna 
+Danae 
+Berneice 
+Annice 
+Alix 
+Verena 
+Verdie 
+Tristan 
+Shawnna 
+Shawana 
+Shaunna 
+Rozella 
+Randee 
+Ranae 
+Milagro 
+Lynell 
+Luise 
+Louie 
+Loida 
+Lisbeth 
+Karleen 
+Junita 
+Jona 
+Isis 
+Hyacinth 
+Hedy 
+Gwenn 
+Ethelene 
+Erline 
+Edward 
+Donya 
+Domonique 
+Delicia 
+Dannette 
+Cicely 
+Branda 
+Blythe 
+Bethann 
+Ashlyn 
+Annalee 
+Alline 
+Yuko 
+Vella 
+Trang 
+Towanda 
+Tesha 
+Sherlyn 
+Narcisa 
+Miguelina 
+Meri 
+Maybell 
+Marlana 
+Marguerita 
+Madlyn 
+Luna 
+Lory 
+Loriann 
+Liberty 
+Leonore 
+Leighann 
+Laurice 
+Latesha 
+Laronda 
+Katrice 
+Kasie 
+Karl 
+Kaley 
+Jadwiga 
+Glennie 
+Gearldine 
+Francina 
+Epifania 
+Dyan 
+Dorie 
+Diedre 
+Denese 
+Demetrice 
+Delena 
+Darby 
+Cristie 
+Cleora 
+Catarina 
+Carisa 
+Bernie 
+Barbera 
+Almeta 
+Trula 
+Tereasa 
+Solange 
+Sheilah 
+Shavonne 
+Sanora 
+Rochell 
+Mathilde 
+Margareta 
+Maia 
+Lynsey 
+Lawanna 
+Launa 
+Kena 
+Keena 
+Katia 
+Jamey 
+Glynda 
+Gaylene 
+Elvina 
+Elanor 
+Danuta 
+Danika 
+Cristen 
+Cordie 
+Coletta 
+Clarita 
+Carmon 
+Brynn 
+Azucena 
+Aundrea 
+Angele 
+Yi 
+Walter 
+Verlie 
+Verlene 
+Tamesha 
+Silvana 
+Sebrina 
+Samira 
+Reda 
+Raylene 
+Penni 
+Pandora 
+Norah 
+Noma 
+Mireille 
+Melissia 
+Maryalice 
+Laraine 
+Kimbery 
+Karyl 
+Karine 
+Kam 
+Jolanda 
+Johana 
+Jesusa 
+Jaleesa 
+Jae 
+Jacquelyne 
+Irish 
+Iluminada 
+Hilaria 
+Hanh 
+Gennie 
+Francie 
+Floretta 
+Exie 
+Edda 
+Drema 
+Delpha 
+Bev 
+Barbar 
+Assunta 
+Ardell 
+Annalisa 
+Alisia 
+Yukiko 
+Yolando 
+Wonda 
+Wei 
+Waltraud 
+Veta 
+Tequila 
+Temeka 
+Tameika 
+Shirleen 
+Shenita 
+Piedad 
+Ozella 
+Mirtha 
+Marilu 
+Kimiko 
+Juliane 
+Jenice 
+Jen 
+Janay 
+Jacquiline 
+Hilde 
+Fe 
+Fae 
+Evan 
+Eugene 
+Elois 
+Echo 
+Devorah 
+Chau 
+Brinda 
+Betsey 
+Arminda 
+Aracelis 
+Apryl 
+Annett 
+Alishia 
+Veola 
+Usha 
+Toshiko 
+Theola 
+Tashia 
+Talitha 
+Shery 
+Rudy 
+Renetta 
+Reiko 
+Rasheeda 
+Omega 
+Obdulia 
+Mika 
+Melaine 
+Meggan 
+Martin 
+Marlen 
+Marget 
+Marceline 
+Mana 
+Magdalen 
+Librada 
+Lezlie 
+Lexie 
+Latashia 
+Lasandra 
+Kelle 
+Isidra 
+Isa 
+Inocencia 
+Gwyn 
+Francoise 
+Erminia 
+Erinn 
+Dimple 
+Devora 
+Criselda 
+Armanda 
+Arie 
+Ariane 
+Angelo 
+Angelena 
+Allen 
+Aliza 
+Adriene 
+Adaline 
+Xochitl 
+Twanna 
+Tran 
+Tomiko 
+Tamisha 
+Taisha 
+Susy 
+Siu 
+Rutha 
+Roxy 
+Rhona 
+Raymond 
+Otha 
+Noriko 
+Natashia 
+Merrie 
+Melvin 
+Marinda 
+Mariko 
+Margert 
+Loris 
+Lizzette 
+Leisha 
+Kaila 
+Ka 
+Joannie 
+Jerrica 
+Jene 
+Jannet 
+Janee 
+Jacinda 
+Herta 
+Elenore 
+Doretta 
+Delaine 
+Daniell 
+Claudie 
+China 
+Britta 
+Apolonia 
+Amberly 
+Alease 
+Yuri 
+Yuk 
+Wen 
+Waneta 
+Ute 
+Tomi 
+Sharri 
+Sandie 
+Roselle 
+Reynalda 
+Raguel 
+Phylicia 
+Patria 
+Olimpia 
+Odelia 
+Mitzie 
+Mitchell 
+Miss 
+Minda 
+Mignon 
+Mica 
+Mendy 
+Marivel 
+Maile 
+Lynetta 
+Lavette 
+Lauryn 
+Latrisha 
+Lakiesha 
+Kiersten 
+Kary 
+Josphine 
+Jolyn 
+Jetta 
+Janise 
+Jacquie 
+Ivelisse 
+Glynis 
+Gianna 
+Gaynelle 
+Emerald 
+Demetrius 
+Danyell 
+Danille 
+Dacia 
+Coralee 
+Cher 
+Ceola 
+Brett 
+Bell 
+Arianne 
+Aleshia 
+Yung 
+Williemae 
+Troy 
+Trinh 
+Thora 
+Tai 
+Svetlana 
+Sherika 
+Shemeka 
+Shaunda 
+Roseline 
+Ricki 
+Melda 
+Mallie 
+Lavonna 
+Latina 
+Larry 
+Laquanda 
+Lala 
+Lachelle 
+Klara 
+Kandis 
+Johna 
+Jeanmarie 
+Jaye 
+Hang 
+Grayce 
+Gertude 
+Emerita 
+Ebonie 
+Clorinda 
+Ching 
+Chery 
+Carola 
+Breann 
+Blossom 
+Bernardine 
+Becki 
+Arletha 
+Argelia 
+Ara 
+Alita 
+Yulanda 
+Yon 
+Yessenia 
+Tobi 
+Tasia 
+Sylvie 
+Shirl 
+Shirely 
+Sheridan 
+Shella 
+Shantelle 
+Sacha 
+Royce 
+Rebecka 
+Reagan 
+Providencia 
+Paulene 
+Misha 
+Miki 
+Marline 
+Marica 
+Lorita 
+Latoyia 
+Lasonya 
+Kerstin 
+Kenda 
+Keitha 
+Kathrin 
+Jaymie 
+Jack 
+Gricelda 
+Ginette 
+Eryn 
+Elina 
+Elfrieda 
+Danyel 
+Cheree 
+Chanelle 
+Barrie 
+Avery 
+Aurore 
+Annamaria 
+Alleen 
+Ailene 
+Aide 
+Yasmine 
+Vashti 
+Valentine 
+Treasa 
+Tory 
+Tiffaney 
+Sheryll 
+Sharie 
+Shanae 
+Sau 
+Raisa 
+Pa 
+Neda 
+Mitsuko 
+Mirella 
+Milda 
+Maryanna 
+Maragret 
+Mabelle 
+Luetta 
+Lorina 
+Letisha 
+Latarsha 
+Lanelle 
+Lajuana 
+Krissy 
+Karly 
+Karena 
+Jon 
+Jessika 
+Jerica 
+Jeanelle 
+January 
+Jalisa 
+Jacelyn 
+Izola 
+Ivey 
+Gregory 
+Euna 
+Etha 
+Drew 
+Domitila 
+Dominica 
+Daina 
+Creola 
+Carli 
+Camie 
+Bunny 
+Brittny 
+Ashanti 
+Anisha 
+Aleen 
+Adah 
+Yasuko 
+Winter 
+Viki 
+Valrie 
+Tona 
+Tinisha 
+Thi 
+Terisa 
+Tatum 
+Taneka 
+Simonne 
+Shalanda 
+Serita 
+Ressie 
+Refugia 
+Paz 
+Olene 
+Na 
+Merrill 
+Margherita 
+Mandie 
+Man 
+Maire 
+Lyndia 
+Luci 
+Lorriane 
+Loreta 
+Leonia 
+Lavona 
+Lashawnda 
+Lakia 
+Kyoko 
+Krystina 
+Krysten 
+Kenia 
+Kelsi 
+Jude 
+Jeanice 
+Isobel 
+Georgiann 
+Genny 
+Felicidad 
+Eilene 
+Deon 
+Deloise 
+Deedee 
+Dannie 
+Conception 
+Clora 
+Cherilyn 
+Chang 
+Calandra 
+Berry 
+Armandina 
+Anisa 
+Ula 
+Timothy 
+Tiera 
+Theressa 
+Stephania 
+Sima 
+Shyla 
+Shonta 
+Shera 
+Shaquita 
+Shala 
+Sammy 
+Rossana 
+Nohemi 
+Nery 
+Moriah 
+Melita 
+Melida 
+Melani 
+Marylynn 
+Marisha 
+Mariette 
+Malorie 
+Madelene 
+Ludivina 
+Loria 
+Lorette 
+Loralee 
+Lianne 
+Leon 
+Lavenia 
+Laurinda 
+Lashon 
+Kit 
+Kimi 
+Keila 
+Katelynn 
+Kai 
+Jone 
+Joane 
+Ji 
+Jayna 
+Janella 
+Ja 
+Hue 
+Hertha 
+Francene 
+Elinore 
+Despina 
+Delsie 
+Deedra 
+Clemencia 
+Carry 
+Carolin 
+Carlos 
+Bulah 
+Brittanie 
+Bok 
+Blondell 
+Bibi 
+Beaulah 
+Beata 
+Annita 
+Agripina 
+Virgen 
+Valene 
+Un 
+Twanda 
+Tommye 
+Toi 
+Tarra 
+Tari 
+Tammera 
+Shakia 
+Sadye 
+Ruthanne 
+Rochel 
+Rivka 
+Pura 
+Nenita 
+Natisha 
+Ming 
+Merrilee 
+Melodee 
+Marvis 
+Lucilla 
+Leena 
+Laveta 
+Larita 
+Lanie 
+Keren 
+Ileen 
+Georgeann 
+Genna 
+Genesis 
+Frida 
+Ewa 
+Eufemia 
+Emely 
+Ela 
+Edyth 
+Deonna 
+Deadra 
+Darlena 
+Chanell 
+Chan 
+Cathern 
+Cassondra 
+Cassaundra 
+Bernarda 
+Berna 
+Arlinda 
+Anamaria 
+Albert 
+Wesley 
+Vertie 
+Valeri 
+Torri 
+Tatyana 
+Stasia 
+Sherise 
+Sherill 
+Season 
+Scottie 
+Sanda 
+Ruthe 
+Rosy 
+Roberto 
+Robbi 
+Ranee 
+Quyen 
+Pearly 
+Palmira 
+Onita 
+Nisha 
+Niesha 
+Nida 
+Nevada 
+Nam 
+Merlyn 
+Mayola 
+Marylouise 
+Maryland 
+Marx 
+Marth 
+Margene 
+Madelaine 
+Londa 
+Leontine 
+Leoma 
+Leia 
+Lawrence 
+Lauralee 
+Lanora 
+Lakita 
+Kiyoko 
+Keturah 
+Katelin 
+Kareen 
+Jonie 
+Johnette 
+Jenee 
+Jeanett 
+Izetta 
+Hiedi 
+Heike 
+Hassie 
+Harold 
+Giuseppina 
+Georgann 
+Fidela 
+Fernande 
+Elwanda 
+Ellamae 
+Eliz 
+Dusti 
+Dotty 
+Cyndy 
+Coralie 
+Celesta 
+Argentina 
+Alverta 
+Xenia 
+Wava 
+Vanetta 
+Torrie 
+Tashina 
+Tandy 
+Tambra 
+Tama 
+Stepanie 
+Shila 
+Shaunta 
+Sharan 
+Shaniqua 
+Shae 
+Setsuko 
+Serafina 
+Sandee 
+Rosamaria 
+Priscila 
+Olinda 
+Nadene 
+Muoi 
+Michelina 
+Mercedez 
+Maryrose 
+Marin 
+Marcene 
+Mao 
+Magali 
+Mafalda 
+Logan 
+Linn 
+Lannie 
+Kayce 
+Karoline 
+Kamilah 
+Kamala 
+Justa 
+Joline 
+Jennine 
+Jacquetta 
+Iraida 
+Gerald 
+Georgeanna 
+Franchesca 
+Fairy 
+Emeline 
+Elane 
+Ehtel 
+Earlie 
+Dulcie 
+Dalene 
+Cris 
+Classie 
+Chere 
+Charis 
+Caroyln 
+Carmina 
+Carita 
+Brian 
+Bethanie 
+Ayako 
+Arica 
+An 
+Alysa 
+Alessandra 
+Akilah 
+Adrien 
+Zetta 
+Youlanda 
+Yelena 
+Yahaira 
+Xuan 
+Wendolyn 
+Victor 
+Tijuana 
+Terrell 
+Terina 
+Teresia 
+Suzi 
+Sunday 
+Sherell 
+Shavonda 
+Shaunte 
+Sharda 
+Shakita 
+Sena 
+Ryann 
+Rubi 
+Riva 
+Reginia 
+Rea 
+Rachal 
+Parthenia 
+Pamula 
+Monnie 
+Monet 
+Michaele 
+Melia 
+Marine 
+Malka 
+Maisha 
+Lisandra 
+Leo 
+Lekisha 
+Lean 
+Laurence 
+Lakendra 
+Krystin 
+Kortney 
+Kizzie 
+Kittie 
+Kera 
+Kendal 
+Kemberly 
+Kanisha 
+Julene 
+Jule 
+Joshua 
+Johanne 
+Jeffrey 
+Jamee 
+Han 
+Halley 
+Gidget 
+Galina 
+Fredricka 
+Fleta 
+Fatimah 
+Eusebia 
+Elza 
+Eleonore 
+Dorthey 
+Doria 
+Donella 
+Dinorah 
+Delorse 
+Claretha 
+Christinia 
+Charlyn 
+Bong 
+Belkis 
+Azzie 
+Andera 
+Aiko 
+Adena 
+Yer 
+Yajaira 
+Wan 
+Vania 
+Ulrike 
+Toshia 
+Tifany 
+Stefany 
+Shizue 
+Shenika 
+Shawanna 
+Sharolyn 
+Sharilyn 
+Shaquana 
+Shantay 
+See 
+Rozanne 
+Roselee 
+Rickie 
+Remona 
+Reanna 
+Raelene 
+Quinn 
+Phung 
+Petronila 
+Natacha 
+Nancey 
+Myrl 
+Miyoko 
+Miesha 
+Merideth 
+Marvella 
+Marquitta 
+Marhta 
+Marchelle 
+Lizeth 
+Libbie 
+Lahoma 
+Ladawn 
+Kina 
+Katheleen 
+Katharyn 
+Karisa 
+Kaleigh 
+Junie 
+Julieann 
+Johnsie 
+Janean 
+Jaimee 
+Jackqueline 
+Hisako 
+Herma 
+Helaine 
+Gwyneth 
+Glenn 
+Gita 
+Eustolia 
+Emelina 
+Elin 
+Edris 
+Donnette 
+Donnetta 
+Dierdre 
+Denae 
+Darcel 
+Claude 
+Clarisa 
+Cinderella 
+Chia 
+Charlesetta 
+Charita 
+Celsa 
+Cassy 
+Cassi 
+Carlee 
+Bruna 
+Brittaney 
+Brande 
+Billi 
+Bao 
+Antonetta 
+Angla 
+Angelyn 
+Analisa 
+Alane 
+Wenona 
+Wendie 
+Veronique 
+Vannesa 
+Tobie 
+Tempie 
+Sumiko 
+Sulema 
+Sparkle 
+Somer 
+Sheba 
+Shayne 
+Sharice 
+Shanel 
+Shalon 
+Sage 
+Roy 
+Rosio 
+Roselia 
+Renay 
+Rema 
+Reena 
+Porsche 
+Ping 
+Peg 
+Ozie 
+Oretha 
+Oralee 
+Oda 
+Nu 
+Ngan 
+Nakesha 
+Milly 
+Marybelle 
+Marlin 
+Maris 
+Margrett 
+Maragaret 
+Manie 
+Lurlene 
+Lillia 
+Lieselotte 
+Lavelle 
+Lashaunda 
+Lakeesha 
+Keith 
+Kaycee 
+Kalyn 
+Joya 
+Joette 
+Jenae 
+Janiece 
+Illa 
+Grisel 
+Glayds 
+Genevie 
+Gala 
+Fredda 
+Fred 
+Elmer 
+Eleonor 
+Debera 
+Deandrea 
+Dan 
+Corrinne 
+Cordia 
+Contessa 
+Colene 
+Cleotilde 
+Charlott 
+Chantay 
+Cecille 
+Beatris 
+Azalee 
+Arlean 
+Ardath 
+Anjelica 
+Anja 
+Alfredia 
+Aleisha 
+Adam 
+Zada 
+Yuonne 
+Xiao 
+Willodean 
+Whitley 
+Vennie 
+Vanna 
+Tyisha 
+Tova 
+Torie 
+Tonisha 
+Tilda 
+Tien 
+Temple 
+Sirena 
+Sherril 
+Shanti 
+Shan 
+Senaida 
+Samella 
+Robbyn 
+Renda 
+Reita 
+Phebe 
+Paulita 
+Nobuko 
+Nguyet 
+Neomi 
+Moon 
+Mikaela 
+Melania 
+Maximina 
+Marg 
+Maisie 
+Lynna 
+Lilli 
+Layne 
+Lashaun 
+Lakenya 
+Lael 
+Kirstie 
+Kathline 
+Kasha 
+Karlyn 
+Karima 
+Jovan 
+Josefine 
+Jennell 
+Jacqui 
+Jackelyn 
+Hyo 
+Hien 
+Grazyna 
+Florrie 
+Floria 
+Eleonora 
+Dwana 
+Dorla 
+Dong 
+Delmy 
+Deja 
+Dede 
+Dann 
+Crysta 
+Clelia 
+Claris 
+Clarence 
+Chieko 
+Cherlyn 
+Cherelle 
+Charmain 
+Chara 
+Cammy 
+Bee 
+Arnette 
+Ardelle 
+Annika 
+Amiee 
+Amee 
+Allena 
+Yvone 
+Yuki 
+Yoshie 
+Yevette 
+Yael 
+Willetta 
+Voncile 
+Venetta 
+Tula 
+Tonette 
+Timika 
+Temika 
+Telma 
+Teisha 
+Taren 
+Ta 
+Stacee 
+Shin 
+Shawnta 
+Saturnina 
+Ricarda 
+Pok 
+Pasty 
+Onie 
+Nubia 
+Mora 
+Mike 
+Marielle 
+Mariella 
+Marianela 
+Mardell 
+Many 
+Luanna 
+Loise 
+Lisabeth 
+Lindsy 
+Lilliana 
+Lilliam 
+Lelah 
+Leigha 
+Leanora 
+Lang 
+Kristeen 
+Khalilah 
+Keeley 
+Kandra 
+Junko 
+Joaquina 
+Jerlene 
+Jani 
+Jamika 
+Jame 
+Hsiu 
+Hermila 
+Golden 
+Genevive 
+Evia 
+Eugena 
+Emmaline 
+Elfreda 
+Elene 
+Donette 
+Delcie 
+Deeanna 
+Darcey 
+Cuc 
+Clarinda 
+Cira 
+Chae 
+Celinda 
+Catheryn 
+Catherin 
+Casimira 
+Carmelia 
+Camellia 
+Breana 
+Bobette 
+Bernardina 
+Bebe 
+Basilia 
+Arlyne 
+Amal 
+Alayna 
+Zonia 
+Zenia 
+Yuriko 
+Yaeko 
+Wynell 
+Willow 
+Willena 
+Vernia 
+Tu 
+Travis 
+Tora 
+Terrilyn 
+Terica 
+Tenesha 
+Tawna 
+Tajuana 
+Taina 
+Stephnie 
+Sona 
+Sol 
+Sina 
+Shondra 
+Shizuko 
+Sherlene 
+Sherice 
+Sharika 
+Rossie 
+Rosena 
+Rory 
+Rima 
+Ria 
+Rheba 
+Renna 
+Peter 
+Natalya 
+Nancee 
+Melodi 
+Meda 
+Maxima 
+Matha 
+Marketta 
+Maricruz 
+Marcelene 
+Malvina 
+Luba 
+Louetta 
+Leida 
+Lecia 
+Lauran 
+Lashawna 
+Laine 
+Khadijah 
+Katerine 
+Kasi 
+Kallie 
+Julietta 
+Jesusita 
+Jestine 
+Jessia 
+Jeremy 
+Jeffie 
+Janyce 
+Isadora 
+Georgianne 
+Fidelia 
+Evita 
+Eura 
+Eulah 
+Estefana 
+Elsy 
+Elizabet 
+Eladia 
+Dodie 
+Dion 
+Dia 
+Denisse 
+Deloras 
+Delila 
+Daysi 
+Dakota 
+Curtis 
+Crystle 
+Concha 
+Colby 
+Claretta 
+Chu 
+Christia 
+Charlsie 
+Charlena 
+Carylon 
+Bettyann 
+Asley 
+Ashlea 
+Amira 
+Ai 
+Agueda 
+Agnus 
+Yuette 
+Vinita 
+Victorina 
+Tynisha 
+Treena 
+Toccara 
+Tish 
+Thomasena 
+Tegan 
+Soila 
+Shiloh 
+Shenna 
+Sharmaine 
+Shantae 
+Shandi 
+September 
+Saran 
+Sarai 
+Sana 
+Samuel 
+Salley 
+Rosette 
+Rolande 
+Regine 
+Otelia 
+Oscar 
+Olevia 
+Nicholle 
+Necole 
+Naida 
+Myrta 
+Myesha 
+Mitsue 
+Minta 
+Mertie 
+Margy 
+Mahalia 
+Madalene 
+Love 
+Loura 
+Lorean 
+Lewis 
+Lesha 
+Leonida 
+Lenita 
+Lavone 
+Lashell 
+Lashandra 
+Lamonica 
+Kimbra 
+Katherina 
+Karry 
+Kanesha 
+Julio 
+Jong 
+Jeneva 
+Jaquelyn 
+Hwa 
+Gilma 
+Ghislaine 
+Gertrudis 
+Fransisca 
+Fermina 
+Ettie 
+Etsuko 
+Ellis 
+Ellan 
+Elidia 
+Edra 
+Dorethea 
+Doreatha 
+Denyse 
+Denny 
+Deetta 
+Daine 
+Cyrstal 
+Corrin 
+Cayla 
+Carlita 
+Camila 
+Burma 
+Bula 
+Buena 
+Blake 
+Barabara 
+Avril 
+Austin 
+Alaine 
+Zana 
+Wilhemina 
+Wanetta 
+Virgil 
+Vi 
+Veronika 
+Vernon 
+Verline 
+Vasiliki 
+Tonita 
+Tisa 
+Teofila 
+Tayna 
+Taunya 
+Tandra 
+Takako 
+Sunni 
+Suanne 
+Sixta 
+Sharell 
+Seema 
+Russell 
+Rosenda 
+Robena 
+Raymonde 
+Pei 
+Pamila 
+Ozell 
+Neida 
+Neely 
+Mistie 
+Micha 
+Merissa 
+Maurita 
+Maryln 
+Maryetta 
+Marshall 
+Marcell 
+Malena 
+Makeda 
+Maddie 
+Lovetta 
+Lourie 
+Lorrine 
+Lorilee 
+Lester 
+Laurena 
+Lashay 
+Larraine 
+Laree 
+Lacresha 
+Kristle 
+Krishna 
+Keva 
+Keira 
+Karole 
+Joie 
+Jinny 
+Jeannetta 
+Jama 
+Heidy 
+Gilberte 
+Gema 
+Faviola 
+Evelynn 
+Enda 
+Elli 
+Ellena 
+Divina 
+Dagny 
+Collene 
+Codi 
+Cindie 
+Chassidy 
+Chasidy 
+Catrice 
+Catherina 
+Cassey 
+Caroll 
+Carlena 
+Candra 
+Calista 
+Bryanna 
+Britteny 
+Beula 
+Bari 
+Audrie 
+Audria 
+Ardelia 
+Annelle 
+Angila 
+Alona 
+Allyn 
+James 
+John 
+Robert 
+Michael 
+William 
+David 
+Richard 
+Charles 
+Joseph 
+Thomas 
+Christopher 
+Daniel 
+Paul 
+Mark 
+Donald 
+George 
+Kenneth 
+Steven 
+Edward 
+Brian 
+Ronald 
+Anthony 
+Kevin 
+Jason 
+Matthew 
+Gary 
+Timothy 
+Jose 
+Larry 
+Jeffrey 
+Frank 
+Scott 
+Eric 
+Stephen 
+Andrew 
+Raymond 
+Gregory 
+Joshua 
+Jerry 
+Dennis 
+Walter 
+Patrick 
+Peter 
+Harold 
+Douglas 
+Henry 
+Carl 
+Arthur 
+Ryan 
+Roger 
+Joe 
+Juan 
+Jack 
+Albert 
+Jonathan 
+Justin 
+Terry 
+Gerald 
+Keith 
+Samuel 
+Willie 
+Ralph 
+Lawrence 
+Nicholas 
+Roy 
+Benjamin 
+Bruce 
+Brandon 
+Adam 
+Harry 
+Fred 
+Wayne 
+Billy 
+Steve 
+Louis 
+Jeremy 
+Aaron 
+Randy 
+Howard 
+Eugene 
+Carlos 
+Russell 
+Bobby 
+Victor 
+Martin 
+Ernest 
+Phillip 
+Todd 
+Jesse 
+Craig 
+Alan 
+Shawn 
+Clarence 
+Sean 
+Philip 
+Chris 
+Johnny 
+Earl 
+Jimmy 
+Antonio 
+Danny 
+Bryan 
+Tony 
+Luis 
+Mike 
+Stanley 
+Leonard 
+Nathan 
+Dale 
+Manuel 
+Rodney 
+Curtis 
+Norman 
+Allen 
+Marvin 
+Vincent 
+Glenn 
+Jeffery 
+Travis 
+Jeff 
+Chad 
+Jacob 
+Lee 
+Melvin 
+Alfred 
+Kyle 
+Francis 
+Bradley 
+Jesus 
+Herbert 
+Frederick 
+Ray 
+Joel 
+Edwin 
+Don 
+Eddie 
+Ricky 
+Troy 
+Randall 
+Barry 
+Alexander 
+Bernard 
+Mario 
+Leroy 
+Francisco 
+Marcus 
+Micheal 
+Theodore 
+Clifford 
+Miguel 
+Oscar 
+Jay 
+Jim 
+Tom 
+Calvin 
+Alex 
+Jon 
+Ronnie 
+Bill 
+Lloyd 
+Tommy 
+Leon 
+Derek 
+Warren 
+Darrell 
+Jerome 
+Floyd 
+Leo 
+Alvin 
+Tim 
+Wesley 
+Gordon 
+Dean 
+Greg 
+Jorge 
+Dustin 
+Pedro 
+Derrick 
+Dan 
+Lewis 
+Zachary 
+Corey 
+Herman 
+Maurice 
+Vernon 
+Roberto 
+Clyde 
+Glen 
+Hector 
+Shane 
+Ricardo 
+Sam 
+Rick 
+Lester 
+Brent 
+Ramon 
+Charlie 
+Tyler 
+Gilbert 
+Gene 
+Marc 
+Reginald 
+Ruben 
+Brett 
+Angel 
+Nathaniel 
+Rafael 
+Leslie 
+Edgar 
+Milton 
+Raul 
+Ben 
+Chester 
+Cecil 
+Duane 
+Franklin 
+Andre 
+Elmer 
+Brad 
+Gabriel 
+Ron 
+Mitchell 
+Roland 
+Arnold 
+Harvey 
+Jared 
+Adrian 
+Karl 
+Cory 
+Claude 
+Erik 
+Darryl 
+Jamie 
+Neil 
+Jessie 
+Christian 
+Javier 
+Fernando 
+Clinton 
+Ted 
+Mathew 
+Tyrone 
+Darren 
+Lonnie 
+Lance 
+Cody 
+Julio 
+Kelly 
+Kurt 
+Allan 
+Nelson 
+Guy 
+Clayton 
+Hugh 
+Max 
+Dwayne 
+Dwight 
+Armando 
+Felix 
+Jimmie 
+Everett 
+Jordan 
+Ian 
+Wallace 
+Ken 
+Bob 
+Jaime 
+Casey 
+Alfredo 
+Alberto 
+Dave 
+Ivan 
+Johnnie 
+Sidney 
+Byron 
+Julian 
+Isaac 
+Morris 
+Clifton 
+Willard 
+Daryl 
+Ross 
+Virgil 
+Andy 
+Marshall 
+Salvador 
+Perry 
+Kirk 
+Sergio 
+Marion 
+Tracy 
+Seth 
+Kent 
+Terrance 
+Rene 
+Eduardo 
+Terrence 
+Enrique 
+Freddie 
+Wade 
+Austin 
+Stuart 
+Fredrick 
+Arturo 
+Alejandro 
+Jackie 
+Joey 
+Nick 
+Luther 
+Wendell 
+Jeremiah 
+Evan 
+Julius 
+Dana 
+Donnie 
+Otis 
+Shannon 
+Trevor 
+Oliver 
+Luke 
+Homer 
+Gerard 
+Doug 
+Kenny 
+Hubert 
+Angelo 
+Shaun 
+Lyle 
+Matt 
+Lynn 
+Alfonso 
+Orlando 
+Rex 
+Carlton 
+Ernesto 
+Cameron 
+Neal 
+Pablo 
+Lorenzo 
+Omar 
+Wilbur 
+Blake 
+Grant 
+Horace 
+Roderick 
+Kerry 
+Abraham 
+Willis 
+Rickey 
+Jean 
+Ira 
+Andres 
+Cesar 
+Johnathan 
+Malcolm 
+Rudolph 
+Damon 
+Kelvin 
+Rudy 
+Preston 
+Alton 
+Archie 
+Marco 
+Wm 
+Pete 
+Randolph 
+Garry 
+Geoffrey 
+Jonathon 
+Felipe 
+Bennie 
+Gerardo 
+Ed 
+Dominic 
+Robin 
+Loren 
+Delbert 
+Colin 
+Guillermo 
+Earnest 
+Lucas 
+Benny 
+Noel 
+Spencer 
+Rodolfo 
+Myron 
+Edmund 
+Garrett 
+Salvatore 
+Cedric 
+Lowell 
+Gregg 
+Sherman 
+Wilson 
+Devin 
+Sylvester 
+Kim 
+Roosevelt 
+Israel 
+Jermaine 
+Forrest 
+Wilbert 
+Leland 
+Simon 
+Guadalupe 
+Clark 
+Irving 
+Carroll 
+Bryant 
+Owen 
+Rufus 
+Woodrow 
+Sammy 
+Kristopher 
+Mack 
+Levi 
+Marcos 
+Gustavo 
+Jake 
+Lionel 
+Marty 
+Taylor 
+Ellis 
+Dallas 
+Gilberto 
+Clint 
+Nicolas 
+Laurence 
+Ismael 
+Orville 
+Drew 
+Jody 
+Ervin 
+Dewey 
+Al 
+Wilfred 
+Josh 
+Hugo 
+Ignacio 
+Caleb 
+Tomas 
+Sheldon 
+Erick 
+Frankie 
+Stewart 
+Doyle 
+Darrel 
+Rogelio 
+Terence 
+Santiago 
+Alonzo 
+Elias 
+Bert 
+Elbert 
+Ramiro 
+Conrad 
+Pat 
+Noah 
+Grady 
+Phil 
+Cornelius 
+Lamar 
+Rolando 
+Clay 
+Percy 
+Dexter 
+Bradford 
+Merle 
+Darin 
+Amos 
+Terrell 
+Moses 
+Irvin 
+Saul 
+Roman 
+Darnell 
+Randal 
+Tommie 
+Timmy 
+Darrin 
+Winston 
+Brendan 
+Toby 
+Van 
+Abel 
+Dominick 
+Boyd 
+Courtney 
+Jan 
+Emilio 
+Elijah 
+Cary 
+Domingo 
+Santos 
+Aubrey 
+Emmett 
+Marlon 
+Emanuel 
+Jerald 
+Edmond 
+Emil 
+Dewayne 
+Will 
+Otto 
+Teddy 
+Reynaldo 
+Bret 
+Morgan 
+Jess 
+Trent 
+Humberto 
+Emmanuel 
+Stephan 
+Louie 
+Vicente 
+Lamont 
+Stacy 
+Garland 
+Miles 
+Micah 
+Efrain 
+Billie 
+Logan 
+Heath 
+Rodger 
+Harley 
+Demetrius 
+Ethan 
+Eldon 
+Rocky 
+Pierre 
+Junior 
+Freddy 
+Eli 
+Bryce 
+Antoine 
+Robbie 
+Kendall 
+Royce 
+Sterling 
+Mickey 
+Chase 
+Grover 
+Elton 
+Cleveland 
+Dylan 
+Chuck 
+Damian 
+Reuben 
+Stan 
+August 
+Leonardo 
+Jasper 
+Russel 
+Erwin 
+Benito 
+Hans 
+Monte 
+Blaine 
+Ernie 
+Curt 
+Quentin 
+Agustin 
+Murray 
+Jamal 
+Devon 
+Adolfo 
+Harrison 
+Tyson 
+Burton 
+Brady 
+Elliott 
+Wilfredo 
+Bart 
+Jarrod 
+Vance 
+Denis 
+Damien 
+Joaquin 
+Harlan 
+Desmond 
+Elliot 
+Darwin 
+Ashley 
+Gregorio 
+Buddy 
+Xavier 
+Kermit 
+Roscoe 
+Esteban 
+Anton 
+Solomon 
+Scotty 
+Norbert 
+Elvin 
+Williams 
+Nolan 
+Carey 
+Rod 
+Quinton 
+Hal 
+Brain 
+Rob 
+Elwood 
+Kendrick 
+Darius 
+Moises 
+Son 
+Marlin 
+Fidel 
+Thaddeus 
+Cliff 
+Marcel 
+Ali 
+Jackson 
+Raphael 
+Bryon 
+Armand 
+Alvaro 
+Jeffry 
+Dane 
+Joesph 
+Thurman 
+Ned 
+Sammie 
+Rusty 
+Michel 
+Monty 
+Rory 
+Fabian 
+Reggie 
+Mason 
+Graham 
+Kris 
+Isaiah 
+Vaughn 
+Gus 
+Avery 
+Loyd 
+Diego 
+Alexis 
+Adolph 
+Norris 
+Millard 
+Rocco 
+Gonzalo 
+Derick 
+Rodrigo 
+Gerry 
+Stacey 
+Carmen 
+Wiley 
+Rigoberto 
+Alphonso 
+Ty 
+Shelby 
+Rickie 
+Noe 
+Vern 
+Bobbie 
+Reed 
+Jefferson 
+Elvis 
+Bernardo 
+Mauricio 
+Hiram 
+Donovan 
+Basil 
+Riley 
+Ollie 
+Nickolas 
+Maynard 
+Scot 
+Vince 
+Quincy 
+Eddy 
+Sebastian 
+Federico 
+Ulysses 
+Heriberto 
+Donnell 
+Cole 
+Denny 
+Davis 
+Gavin 
+Emery 
+Ward 
+Romeo 
+Jayson 
+Dion 
+Dante 
+Clement 
+Coy 
+Odell 
+Maxwell 
+Jarvis 
+Bruno 
+Issac 
+Mary 
+Dudley 
+Brock 
+Sanford 
+Colby 
+Carmelo 
+Barney 
+Nestor 
+Hollis 
+Stefan 
+Donny 
+Art 
+Linwood 
+Beau 
+Weldon 
+Galen 
+Isidro 
+Truman 
+Delmar 
+Johnathon 
+Silas 
+Frederic 
+Dick 
+Kirby 
+Irwin 
+Cruz 
+Merlin 
+Merrill 
+Charley 
+Marcelino 
+Lane 
+Harris 
+Cleo 
+Carlo 
+Trenton 
+Kurtis 
+Hunter 
+Aurelio 
+Winfred 
+Vito 
+Collin 
+Denver 
+Carter 
+Leonel 
+Emory 
+Pasquale 
+Mohammad 
+Mariano 
+Danial 
+Blair 
+Landon 
+Dirk 
+Branden 
+Adan 
+Numbers 
+Clair 
+Buford 
+German 
+Bernie 
+Wilmer 
+Joan 
+Emerson 
+Zachery 
+Fletcher 
+Jacques 
+Errol 
+Dalton 
+Monroe 
+Josue 
+Dominique 
+Edwardo 
+Booker 
+Wilford 
+Sonny 
+Shelton 
+Carson 
+Theron 
+Raymundo 
+Daren 
+Tristan 
+Houston 
+Robby 
+Lincoln 
+Jame 
+Genaro 
+Gale 
+Bennett 
+Octavio 
+Cornell 
+Laverne 
+Hung 
+Arron 
+Antony 
+Herschel 
+Alva 
+Giovanni 
+Garth 
+Cyrus 
+Cyril 
+Ronny 
+Stevie 
+Lon 
+Freeman 
+Erin 
+Duncan 
+Kennith 
+Carmine 
+Augustine 
+Young 
+Erich 
+Chadwick 
+Wilburn 
+Russ 
+Reid 
+Myles 
+Anderson 
+Morton 
+Jonas 
+Forest 
+Mitchel 
+Mervin 
+Zane 
+Rich 
+Jamel 
+Lazaro 
+Alphonse 
+Randell 
+Major 
+Johnie 
+Jarrett 
+Brooks 
+Ariel 
+Abdul 
+Dusty 
+Luciano 
+Lindsey 
+Tracey 
+Seymour 
+Scottie 
+Eugenio 
+Mohammed 
+Sandy 
+Valentin 
+Chance 
+Arnulfo 
+Lucien 
+Ferdinand 
+Thad 
+Ezra 
+Sydney 
+Aldo 
+Rubin 
+Royal 
+Mitch 
+Earle 
+Abe 
+Wyatt 
+Marquis 
+Lanny 
+Kareem 
+Jamar 
+Boris 
+Isiah 
+Emile 
+Elmo 
+Aron 
+Leopoldo 
+Everette 
+Josef 
+Gail 
+Eloy 
+Dorian 
+Rodrick 
+Reinaldo 
+Lucio 
+Jerrod 
+Weston 
+Hershel 
+Barton 
+Parker 
+Lemuel 
+Lavern 
+Burt 
+Jules 
+Gil 
+Eliseo 
+Ahmad 
+Nigel 
+Efren 
+Antwan 
+Alden 
+Margarito 
+Coleman 
+Refugio 
+Dino 
+Osvaldo 
+Les 
+Deandre 
+Normand 
+Kieth 
+Ivory 
+Andrea 
+Trey 
+Norberto 
+Napoleon 
+Jerold 
+Fritz 
+Rosendo 
+Milford 
+Sang 
+Deon 
+Christoper 
+Alfonzo 
+Lyman 
+Josiah 
+Brant 
+Wilton 
+Rico 
+Jamaal 
+Dewitt 
+Carol 
+Brenton 
+Yong 
+Olin 
+Foster 
+Faustino 
+Claudio 
+Judson 
+Gino 
+Edgardo 
+Berry 
+Alec 
+Tanner 
+Jarred 
+Donn 
+Trinidad 
+Tad 
+Shirley 
+Prince 
+Porfirio 
+Odis 
+Maria 
+Lenard 
+Chauncey 
+Chang 
+Tod 
+Mel 
+Marcelo 
+Kory 
+Augustus 
+Keven 
+Hilario 
+Bud 
+Sal 
+Rosario 
+Orval 
+Mauro 
+Dannie 
+Zachariah 
+Olen 
+Anibal 
+Milo 
+Jed 
+Frances 
+Thanh 
+Dillon 
+Amado 
+Newton 
+Connie 
+Lenny 
+Tory 
+Richie 
+Lupe 
+Horacio 
+Brice 
+Mohamed 
+Delmer 
+Dario 
+Reyes 
+Dee 
+Mac 
+Jonah 
+Jerrold 
+Robt 
+Hank 
+Sung 
+Rupert 
+Rolland 
+Kenton 
+Damion 
+Chi 
+Antone 
+Waldo 
+Fredric 
+Bradly 
+Quinn 
+Kip 
+Burl 
+Walker 
+Tyree 
+Jefferey 
+Ahmed 
+Willy 
+Stanford 
+Oren 
+Noble 
+Moshe 
+Mikel 
+Enoch 
+Brendon 
+Quintin 
+Jamison 
+Florencio 
+Darrick 
+Tobias 
+Minh 
+Hassan 
+Giuseppe 
+Demarcus 
+Cletus 
+Tyrell 
+Lyndon 
+Keenan 
+Werner 
+Theo 
+Geraldo 
+Lou 
+Columbus 
+Chet 
+Bertram 
+Markus 
+Huey 
+Hilton 
+Dwain 
+Donte 
+Tyron 
+Omer 
+Isaias 
+Hipolito 
+Fermin 
+Chung 
+Adalberto 
+Valentine 
+Jamey 
+Bo 
+Barrett 
+Whitney 
+Teodoro 
+Mckinley 
+Maximo 
+Garfield 
+Sol 
+Raleigh 
+Lawerence 
+Abram 
+Rashad 
+King 
+Emmitt 
+Daron 
+Chong 
+Samual 
+Paris 
+Otha 
+Miquel 
+Lacy 
+Eusebio 
+Dong 
+Domenic 
+Darron 
+Buster 
+Antonia 
+Wilber 
+Renato 
+Jc 
+Hoyt 
+Haywood 
+Ezekiel 
+Chas 
+Florentino 
+Elroy 
+Clemente 
+Arden 
+Neville 
+Kelley 
+Edison 
+Deshawn 
+Carrol 
+Shayne 
+Nathanial 
+Jordon 
+Danilo 
+Claud 
+Val 
+Sherwood 
+Raymon 
+Rayford 
+Cristobal 
+Ambrose 
+Titus 
+Hyman 
+Felton 
+Ezequiel 
+Erasmo 
+Stanton 
+Lonny 
+Len 
+Ike 
+Milan 
+Lino 
+Jarod 
+Herb 
+Andreas 
+Walton 
+Rhett 
+Palmer 
+Jude 
+Douglass 
+Cordell 
+Oswaldo 
+Ellsworth 
+Virgilio 
+Toney 
+Nathanael 
+Del 
+Britt 
+Benedict 
+Mose 
+Hong 
+Leigh 
+Johnson 
+Isreal 
+Gayle 
+Garret 
+Fausto 
+Asa 
+Arlen 
+Zack 
+Warner 
+Modesto 
+Francesco 
+Manual 
+Jae 
+Gaylord 
+Gaston 
+Filiberto 
+Deangelo 
+Michale 
+Granville 
+Wes 
+Malik 
+Zackary 
+Tuan 
+Nicky 
+Eldridge 
+Cristopher 
+Cortez 
+Antione 
+Malcom 
+Long 
+Korey 
+Jospeh 
+Colton 
+Waylon 
+Von 
+Hosea 
+Shad 
+Santo 
+Rudolf 
+Rolf 
+Rey 
+Renaldo 
+Marcellus 
+Lucius 
+Lesley 
+Kristofer 
+Boyce 
+Benton 
+Man 
+Kasey 
+Jewell 
+Hayden 
+Harland 
+Arnoldo 
+Rueben 
+Leandro 
+Kraig 
+Jerrell 
+Jeromy 
+Hobert 
+Cedrick 
+Arlie 
+Winford 
+Wally 
+Patricia 
+Luigi 
+Keneth 
+Jacinto 
+Graig 
+Franklyn 
+Edmundo 
+Sid 
+Porter 
+Leif 
+Lauren 
+Jeramy 
+Elisha 
+Buck 
+Willian 
+Vincenzo 
+Shon 
+Michal 
+Lynwood 
+Lindsay 
+Jewel 
+Jere 
+Hai 
+Elden 
+Dorsey 
+Darell 
+Broderick 
+Alonso 
diff --git a/hyracks/hyracks-storage-am-common/src/main/resources/dist.all.last.cleaned b/hyracks/hyracks-storage-am-common/src/main/resources/dist.all.last.cleaned
new file mode 100755
index 0000000..c9ec798
--- /dev/null
+++ b/hyracks/hyracks-storage-am-common/src/main/resources/dist.all.last.cleaned
@@ -0,0 +1,88799 @@
+Smith
+Johnson
+Williams
+Jones
+Brown
+Davis
+Miller
+Wilson
+Moore
+Taylor
+Anderson
+Thomas
+Jackson
+White
+Harris
+Martin
+Thompson
+Garcia
+Martinez
+Robinson
+Clark
+Rodriguez
+Lewis
+Lee
+Walker
+Hall
+Allen
+Young
+Hernandez
+King
+Wright
+Lopez
+Hill
+Scott
+Green
+Adams
+Baker
+Gonzalez
+Nelson
+Carter
+Mitchell
+Perez
+Roberts
+Turner
+Phillips
+Campbell
+Parker
+Evans
+Edwards
+Collins
+Stewart
+Sanchez
+Morris
+Rogers
+Reed
+Cook
+Morgan
+Bell
+Murphy
+Bailey
+Rivera
+Cooper
+Richardson
+Cox
+Howard
+Ward
+Torres
+Peterson
+Gray
+Ramirez
+James
+Watson
+Brooks
+Kelly
+Sanders
+Price
+Bennett
+Wood
+Barnes
+Ross
+Henderson
+Coleman
+Jenkins
+Perry
+Powell
+Long
+Patterson
+Hughes
+Flores
+Washington
+Butler
+Simmons
+Foster
+Gonzales
+Bryant
+Alexander
+Russell
+Griffin
+Diaz
+Hayes
+Myers
+Ford
+Hamilton
+Graham
+Sullivan
+Wallace
+Woods
+Cole
+West
+Jordan
+Owens
+Reynolds
+Fisher
+Ellis
+Harrison
+Gibson
+Mcdonald
+Cruz
+Marshall
+Ortiz
+Gomez
+Murray
+Freeman
+Wells
+Webb
+Simpson
+Stevens
+Tucker
+Porter
+Hunter
+Hicks
+Crawford
+Henry
+Boyd
+Mason
+Morales
+Kennedy
+Warren
+Dixon
+Ramos
+Reyes
+Burns
+Gordon
+Shaw
+Holmes
+Rice
+Robertson
+Hunt
+Black
+Daniels
+Palmer
+Mills
+Nichols
+Grant
+Knight
+Ferguson
+Rose
+Stone
+Hawkins
+Dunn
+Perkins
+Hudson
+Spencer
+Gardner
+Stephens
+Payne
+Pierce
+Berry
+Matthews
+Arnold
+Wagner
+Willis
+Ray
+Watkins
+Olson
+Carroll
+Duncan
+Snyder
+Hart
+Cunningham
+Bradley
+Lane
+Andrews
+Ruiz
+Harper
+Fox
+Riley
+Armstrong
+Carpenter
+Weaver
+Greene
+Lawrence
+Elliott
+Chavez
+Sims
+Austin
+Peters
+Kelley
+Franklin
+Lawson
+Fields
+Gutierrez
+Ryan
+Schmidt
+Carr
+Vasquez
+Castillo
+Wheeler
+Chapman
+Oliver
+Montgomery
+Richards
+Williamson
+Johnston
+Banks
+Meyer
+Bishop
+Mccoy
+Howell
+Alvarez
+Morrison
+Hansen
+Fernandez
+Garza
+Harvey
+Little
+Burton
+Stanley
+Nguyen
+George
+Jacobs
+Reid
+Kim
+Fuller
+Lynch
+Dean
+Gilbert
+Garrett
+Romero
+Welch
+Larson
+Frazier
+Burke
+Hanson
+Day
+Mendoza
+Moreno
+Bowman
+Medina
+Fowler
+Brewer
+Hoffman
+Carlson
+Silva
+Pearson
+Holland
+Douglas
+Fleming
+Jensen
+Vargas
+Byrd
+Davidson
+Hopkins
+May
+Terry
+Herrera
+Wade
+Soto
+Walters
+Curtis
+Neal
+Caldwell
+Lowe
+Jennings
+Barnett
+Graves
+Jimenez
+Horton
+Shelton
+Barrett
+Obrien
+Castro
+Sutton
+Gregory
+Mckinney
+Lucas
+Miles
+Craig
+Rodriquez
+Chambers
+Holt
+Lambert
+Fletcher
+Watts
+Bates
+Hale
+Rhodes
+Pena
+Beck
+Newman
+Haynes
+Mcdaniel
+Mendez
+Bush
+Vaughn
+Parks
+Dawson
+Santiago
+Norris
+Hardy
+Love
+Steele
+Curry
+Powers
+Schultz
+Barker
+Guzman
+Page
+Munoz
+Ball
+Keller
+Chandler
+Weber
+Leonard
+Walsh
+Lyons
+Ramsey
+Wolfe
+Schneider
+Mullins
+Benson
+Sharp
+Bowen
+Daniel
+Barber
+Cummings
+Hines
+Baldwin
+Griffith
+Valdez
+Hubbard
+Salazar
+Reeves
+Warner
+Stevenson
+Burgess
+Santos
+Tate
+Cross
+Garner
+Mann
+Mack
+Moss
+Thornton
+Dennis
+Mcgee
+Farmer
+Delgado
+Aguilar
+Vega
+Glover
+Manning
+Cohen
+Harmon
+Rodgers
+Robbins
+Newton
+Todd
+Blair
+Higgins
+Ingram
+Reese
+Cannon
+Strickland
+Townsend
+Potter
+Goodwin
+Walton
+Rowe
+Hampton
+Ortega
+Patton
+Swanson
+Joseph
+Francis
+Goodman
+Maldonado
+Yates
+Becker
+Erickson
+Hodges
+Rios
+Conner
+Adkins
+Webster
+Norman
+Malone
+Hammond
+Flowers
+Cobb
+Moody
+Quinn
+Blake
+Maxwell
+Pope
+Floyd
+Osborne
+Paul
+Mccarthy
+Guerrero
+Lindsey
+Estrada
+Sandoval
+Gibbs
+Tyler
+Gross
+Fitzgerald
+Stokes
+Doyle
+Sherman
+Saunders
+Wise
+Colon
+Gill
+Alvarado
+Greer
+Padilla
+Simon
+Waters
+Nunez
+Ballard
+Schwartz
+Mcbride
+Houston
+Christensen
+Klein
+Pratt
+Briggs
+Parsons
+Mclaughlin
+Zimmerman
+French
+Buchanan
+Moran
+Copeland
+Roy
+Pittman
+Brady
+Mccormick
+Holloway
+Brock
+Poole
+Frank
+Logan
+Owen
+Bass
+Marsh
+Drake
+Wong
+Jefferson
+Park
+Morton
+Abbott
+Sparks
+Patrick
+Norton
+Huff
+Clayton
+Massey
+Lloyd
+Figueroa
+Carson
+Bowers
+Roberson
+Barton
+Tran
+Lamb
+Harrington
+Casey
+Boone
+Cortez
+Clarke
+Mathis
+Singleton
+Wilkins
+Cain
+Bryan
+Underwood
+Hogan
+Mckenzie
+Collier
+Luna
+Phelps
+Mcguire
+Allison
+Bridges
+Wilkerson
+Nash
+Summers
+Atkins
+Wilcox
+Pitts
+Conley
+Marquez
+Burnett
+Richard
+Cochran
+Chase
+Davenport
+Hood
+Gates
+Clay
+Ayala
+Sawyer
+Roman
+Vazquez
+Dickerson
+Hodge
+Acosta
+Flynn
+Espinoza
+Nicholson
+Monroe
+Wolf
+Morrow
+Kirk
+Randall
+Anthony
+Whitaker
+Oconnor
+Skinner
+Ware
+Molina
+Kirby
+Huffman
+Bradford
+Charles
+Gilmore
+Dominguez
+Oneal
+Bruce
+Lang
+Combs
+Kramer
+Heath
+Hancock
+Gallagher
+Gaines
+Shaffer
+Short
+Wiggins
+Mathews
+Mcclain
+Fischer
+Wall
+Small
+Melton
+Hensley
+Bond
+Dyer
+Cameron
+Grimes
+Contreras
+Christian
+Wyatt
+Baxter
+Snow
+Mosley
+Shepherd
+Larsen
+Hoover
+Beasley
+Glenn
+Petersen
+Whitehead
+Meyers
+Keith
+Garrison
+Vincent
+Shields
+Horn
+Savage
+Olsen
+Schroeder
+Hartman
+Woodard
+Mueller
+Kemp
+Deleon
+Booth
+Patel
+Calhoun
+Wiley
+Eaton
+Cline
+Navarro
+Harrell
+Lester
+Humphrey
+Parrish
+Duran
+Hutchinson
+Hess
+Dorsey
+Bullock
+Robles
+Beard
+Dalton
+Avila
+Vance
+Rich
+Blackwell
+York
+Johns
+Blankenship
+Trevino
+Salinas
+Campos
+Pruitt
+Moses
+Callahan
+Golden
+Montoya
+Hardin
+Guerra
+Mcdowell
+Carey
+Stafford
+Gallegos
+Henson
+Wilkinson
+Booker
+Merritt
+Miranda
+Atkinson
+Orr
+Decker
+Hobbs
+Preston
+Tanner
+Knox
+Pacheco
+Stephenson
+Glass
+Rojas
+Serrano
+Marks
+Hickman
+English
+Sweeney
+Strong
+Prince
+Mcclure
+Conway
+Walter
+Roth
+Maynard
+Farrell
+Lowery
+Hurst
+Nixon
+Weiss
+Trujillo
+Ellison
+Sloan
+Juarez
+Winters
+Mclean
+Randolph
+Leon
+Boyer
+Villarreal
+Mccall
+Gentry
+Carrillo
+Kent
+Ayers
+Lara
+Shannon
+Sexton
+Pace
+Hull
+Leblanc
+Browning
+Velasquez
+Leach
+Chang
+House
+Sellers
+Herring
+Noble
+Foley
+Bartlett
+Mercado
+Landry
+Durham
+Walls
+Barr
+Mckee
+Bauer
+Rivers
+Everett
+Bradshaw
+Pugh
+Velez
+Rush
+Estes
+Dodson
+Morse
+Sheppard
+Weeks
+Camacho
+Bean
+Barron
+Livingston
+Middleton
+Spears
+Branch
+Blevins
+Chen
+Kerr
+Mcconnell
+Hatfield
+Harding
+Ashley
+Solis
+Herman
+Frost
+Giles
+Blackburn
+William
+Pennington
+Woodward
+Finley
+Mcintosh
+Koch
+Best
+Solomon
+Mccullough
+Dudley
+Nolan
+Blanchard
+Rivas
+Brennan
+Mejia
+Kane
+Benton
+Joyce
+Buckley
+Haley
+Valentine
+Maddox
+Russo
+Mcknight
+Buck
+Moon
+Mcmillan
+Crosby
+Berg
+Dotson
+Mays
+Roach
+Church
+Chan
+Richmond
+Meadows
+Faulkner
+Oneill
+Knapp
+Kline
+Barry
+Ochoa
+Jacobson
+Gay
+Avery
+Hendricks
+Horne
+Shepard
+Hebert
+Cherry
+Cardenas
+Mcintyre
+Whitney
+Waller
+Holman
+Donaldson
+Cantu
+Terrell
+Morin
+Gillespie
+Fuentes
+Tillman
+Sanford
+Bentley
+Peck
+Key
+Salas
+Rollins
+Gamble
+Dickson
+Battle
+Santana
+Cabrera
+Cervantes
+Howe
+Hinton
+Hurley
+Spence
+Zamora
+Yang
+Mcneil
+Suarez
+Case
+Petty
+Gould
+Mcfarland
+Sampson
+Carver
+Bray
+Rosario
+Macdonald
+Stout
+Hester
+Melendez
+Dillon
+Farley
+Hopper
+Galloway
+Potts
+Bernard
+Joyner
+Stein
+Aguirre
+Osborn
+Mercer
+Bender
+Franco
+Rowland
+Sykes
+Benjamin
+Travis
+Pickett
+Crane
+Sears
+Mayo
+Dunlap
+Hayden
+Wilder
+Mckay
+Coffey
+Mccarty
+Ewing
+Cooley
+Vaughan
+Bonner
+Cotton
+Holder
+Stark
+Ferrell
+Cantrell
+Fulton
+Lynn
+Lott
+Calderon
+Rosa
+Pollard
+Hooper
+Burch
+Mullen
+Fry
+Riddle
+Levy
+David
+Duke
+Odonnell
+Guy
+Michael
+Britt
+Frederick
+Daugherty
+Berger
+Dillard
+Alston
+Jarvis
+Frye
+Riggs
+Chaney
+Odom
+Duffy
+Fitzpatrick
+Valenzuela
+Merrill
+Mayer
+Alford
+Mcpherson
+Acevedo
+Donovan
+Barrera
+Albert
+Cote
+Reilly
+Compton
+Raymond
+Mooney
+Mcgowan
+Craft
+Cleveland
+Clemons
+Wynn
+Nielsen
+Baird
+Stanton
+Snider
+Rosales
+Bright
+Witt
+Stuart
+Hays
+Holden
+Rutledge
+Kinney
+Clements
+Castaneda
+Slater
+Hahn
+Emerson
+Conrad
+Burks
+Delaney
+Pate
+Lancaster
+Sweet
+Justice
+Tyson
+Sharpe
+Whitfield
+Talley
+Macias
+Irwin
+Burris
+Ratliff
+Mccray
+Madden
+Kaufman
+Beach
+Goff
+Cash
+Bolton
+Mcfadden
+Levine
+Good
+Byers
+Kirkland
+Kidd
+Workman
+Carney
+Dale
+Mcleod
+Holcomb
+England
+Finch
+Head
+Burt
+Hendrix
+Sosa
+Haney
+Franks
+Sargent
+Nieves
+Downs
+Rasmussen
+Bird
+Hewitt
+Lindsay
+Le
+Foreman
+Valencia
+Oneil
+Delacruz
+Vinson
+Dejesus
+Hyde
+Forbes
+Gilliam
+Guthrie
+Wooten
+Huber
+Barlow
+Boyle
+Mcmahon
+Buckner
+Rocha
+Puckett
+Langley
+Knowles
+Cooke
+Velazquez
+Whitley
+Noel
+Vang
+Shea
+Rouse
+Hartley
+Mayfield
+Elder
+Rankin
+Hanna
+Cowan
+Lucero
+Arroyo
+Slaughter
+Haas
+Oconnell
+Minor
+Kendrick
+Shirley
+Kendall
+Boucher
+Archer
+Boggs
+Odell
+Dougherty
+Andersen
+Newell
+Crowe
+Wang
+Friedman
+Bland
+Swain
+Holley
+Felix
+Pearce
+Childs
+Yarbrough
+Galvan
+Proctor
+Meeks
+Lozano
+Mora
+Rangel
+Bacon
+Villanueva
+Schaefer
+Rosado
+Helms
+Boyce
+Goss
+Stinson
+Smart
+Lake
+Ibarra
+Hutchins
+Covington
+Reyna
+Gregg
+Werner
+Crowley
+Hatcher
+Mackey
+Bunch
+Womack
+Polk
+Jamison
+Dodd
+Childress
+Childers
+Camp
+Villa
+Dye
+Springer
+Mahoney
+Dailey
+Belcher
+Lockhart
+Griggs
+Costa
+Connor
+Brandt
+Winter
+Walden
+Moser
+Tracy
+Tatum
+Mccann
+Akers
+Lutz
+Pryor
+Law
+Orozco
+Mcallister
+Lugo
+Davies
+Shoemaker
+Madison
+Rutherford
+Newsome
+Magee
+Chamberlain
+Blanton
+Simms
+Godfrey
+Flanagan
+Crum
+Cordova
+Escobar
+Downing
+Sinclair
+Donahue
+Krueger
+Mcginnis
+Gore
+Farris
+Webber
+Corbett
+Andrade
+Starr
+Lyon
+Yoder
+Hastings
+Mcgrath
+Spivey
+Krause
+Harden
+Crabtree
+Kirkpatrick
+Hollis
+Brandon
+Arrington
+Ervin
+Clifton
+Ritter
+Mcghee
+Bolden
+Maloney
+Gagnon
+Dunbar
+Ponce
+Pike
+Mayes
+Heard
+Beatty
+Mobley
+Kimball
+Butts
+Montes
+Herbert
+Grady
+Eldridge
+Braun
+Hamm
+Gibbons
+Seymour
+Moyer
+Manley
+Herron
+Plummer
+Elmore
+Cramer
+Gary
+Rucker
+Hilton
+Blue
+Pierson
+Fontenot
+Field
+Rubio
+Grace
+Goldstein
+Elkins
+Wills
+Novak
+John
+Hickey
+Worley
+Gorman
+Katz
+Dickinson
+Broussard
+Fritz
+Woodruff
+Crow
+Christopher
+Britton
+Forrest
+Nance
+Lehman
+Bingham
+Zuniga
+Whaley
+Shafer
+Coffman
+Steward
+Delarosa
+Nix
+Neely
+Numbers
+Mata
+Manuel
+Davila
+Mccabe
+Kessler
+Emery
+Bowling
+Hinkle
+Welsh
+Pagan
+Goldberg
+Goins
+Crouch
+Cuevas
+Quinones
+Mcdermott
+Hendrickson
+Samuels
+Denton
+Bergeron
+Lam
+Ivey
+Locke
+Haines
+Thurman
+Snell
+Hoskins
+Byrne
+Milton
+Winston
+Arthur
+Arias
+Stanford
+Roe
+Corbin
+Beltran
+Chappell
+Hurt
+Downey
+Dooley
+Tuttle
+Couch
+Payton
+Mcelroy
+Crockett
+Groves
+Clement
+Leslie
+Cartwright
+Dickey
+Mcgill
+Dubois
+Muniz
+Erwin
+Self
+Tolbert
+Dempsey
+Cisneros
+Sewell
+Latham
+Garland
+Vigil
+Tapia
+Sterling
+Rainey
+Norwood
+Lacy
+Stroud
+Meade
+Amos
+Tipton
+Lord
+Kuhn
+Hilliard
+Bonilla
+Teague
+Courtney
+Gunn
+Ho
+Greenwood
+Correa
+Reece
+Weston
+Poe
+Trent
+Pineda
+Phipps
+Frey
+Kaiser
+Ames
+Paige
+Gunter
+Schmitt
+Milligan
+Espinosa
+Carlton
+Bowden
+Vickers
+Lowry
+Pritchard
+Costello
+Piper
+Mcclellan
+Lovell
+Drew
+Sheehan
+Quick
+Hatch
+Dobson
+Singh
+Jeffries
+Hollingsworth
+Sorensen
+Meza
+Fink
+Donnelly
+Burrell
+Bruno
+Tomlinson
+Colbert
+Billings
+Ritchie
+Helton
+Sutherland
+Peoples
+Mcqueen
+Gaston
+Thomason
+Mckinley
+Givens
+Crocker
+Vogel
+Robison
+Dunham
+Coker
+Swartz
+Keys
+Lilly
+Ladner
+Hannah
+Willard
+Richter
+Hargrove
+Edmonds
+Brantley
+Albright
+Murdock
+Boswell
+Muller
+Quintero
+Padgett
+Kenney
+Daly
+Connolly
+Pierre
+Inman
+Quintana
+Lund
+Barnard
+Villegas
+Simons
+Land
+Huggins
+Tidwell
+Sanderson
+Bullard
+Mcclendon
+Duarte
+Draper
+Meredith
+Marrero
+Dwyer
+Abrams
+Stover
+Goode
+Fraser
+Crews
+Bernal
+Smiley
+Godwin
+Fish
+Conklin
+Mcneal
+Baca
+Esparza
+Crowder
+Bower
+Nicholas
+Chung
+Brewster
+Mcneill
+Dick
+Rodrigues
+Leal
+Coates
+Raines
+Mccain
+Mccord
+Miner
+Holbrook
+Swift
+Dukes
+Carlisle
+Aldridge
+Ackerman
+Starks
+Ricks
+Holliday
+Ferris
+Hairston
+Sheffield
+Lange
+Fountain
+Marino
+Doss
+Betts
+Kaplan
+Carmichael
+Bloom
+Ruffin
+Penn
+Kern
+Bowles
+Sizemore
+Larkin
+Dupree
+Jewell
+Silver
+Seals
+Metcalf
+Hutchison
+Henley
+Farr
+Castle
+Mccauley
+Hankins
+Gustafson
+Deal
+Curran
+Ash
+Waddell
+Ramey
+Cates
+Pollock
+Major
+Irvin
+Cummins
+Messer
+Heller
+Dewitt
+Lin
+Funk
+Cornett
+Palacios
+Galindo
+Cano
+Hathaway
+Singer
+Pham
+Enriquez
+Aaron
+Salgado
+Pelletier
+Painter
+Wiseman
+Blount
+Hand
+Feliciano
+Temple
+Houser
+Doherty
+Mead
+Mcgraw
+Toney
+Swan
+Melvin
+Capps
+Blanco
+Blackmon
+Wesley
+Thomson
+Mcmanus
+Fair
+Burkett
+Post
+Gleason
+Rudolph
+Ott
+Dickens
+Cormier
+Voss
+Rushing
+Rosenberg
+Hurd
+Dumas
+Benitez
+Arellano
+Story
+Marin
+Caudill
+Bragg
+Jaramillo
+Huerta
+Gipson
+Colvin
+Biggs
+Vela
+Platt
+Cassidy
+Tompkins
+Mccollum
+Kay
+Gabriel
+Dolan
+Daley
+Crump
+Street
+Sneed
+Kilgore
+Grove
+Grimm
+Davison
+Brunson
+Prater
+Marcum
+Devine
+Kyle
+Dodge
+Stratton
+Rosas
+Choi
+Tripp
+Ledbetter
+Lay
+Hightower
+Haywood
+Feldman
+Epps
+Yeager
+Posey
+Sylvester
+Scruggs
+Cope
+Stubbs
+Richey
+Overton
+Trotter
+Sprague
+Cordero
+Butcher
+Burger
+Stiles
+Burgos
+Woodson
+Horner
+Bassett
+Purcell
+Haskins
+Gee
+Akins
+Abraham
+Hoyt
+Ziegler
+Spaulding
+Hadley
+Grubbs
+Sumner
+Murillo
+Zavala
+Shook
+Lockwood
+Jarrett
+Driscoll
+Dahl
+Thorpe
+Sheridan
+Redmond
+Putnam
+Mcwilliams
+Mcrae
+Cornell
+Felton
+Romano
+Joiner
+Sadler
+Hedrick
+Hager
+Hagen
+Fitch
+Coulter
+Thacker
+Mansfield
+Langston
+Guidry
+Ferreira
+Corley
+Conn
+Rossi
+Lackey
+Cody
+Baez
+Saenz
+Mcnamara
+Darnell
+Michel
+Mcmullen
+Mckenna
+Mcdonough
+Link
+Engel
+Browne
+Roper
+Peacock
+Eubanks
+Drummond
+Stringer
+Pritchett
+Parham
+Mims
+Landers
+Ham
+Grayson
+Stacy
+Schafer
+Egan
+Timmons
+Ohara
+Keen
+Hamlin
+Finn
+Cortes
+Mcnair
+Louis
+Clifford
+Nadeau
+Moseley
+Michaud
+Rosen
+Oakes
+Kurtz
+Jeffers
+Calloway
+Beal
+Bautista
+Winn
+Suggs
+Stern
+Stapleton
+Lyles
+Laird
+Montano
+Diamond
+Dawkins
+Roland
+Hagan
+Goldman
+Bryson
+Barajas
+Lovett
+Segura
+Metz
+Lockett
+Langford
+Hinson
+Eastman
+Rock
+Hooks
+Woody
+Smallwood
+Shapiro
+Crowell
+Whalen
+Triplett
+Hooker
+Chatman
+Aldrich
+Cahill
+Youngblood
+Ybarra
+Stallings
+Sheets
+Samuel
+Reeder
+Person
+Pack
+Lacey
+Connelly
+Bateman
+Abernathy
+Winkler
+Wilkes
+Masters
+Hackett
+Granger
+Gillis
+Schmitz
+Sapp
+Napier
+Souza
+Lanier
+Gomes
+Weir
+Otero
+Ledford
+Burroughs
+Babcock
+Ventura
+Siegel
+Dugan
+Clinton
+Christie
+Bledsoe
+Atwood
+Wray
+Varner
+Spangler
+Otto
+Anaya
+Staley
+Kraft
+Fournier
+Eddy
+Belanger
+Wolff
+Thorne
+Bynum
+Burnette
+Boykin
+Swenson
+Purvis
+Pina
+Khan
+Duvall
+Darby
+Xiong
+Kauffman
+Ali
+Yu
+Healy
+Engle
+Corona
+Benoit
+Valle
+Steiner
+Spicer
+Shaver
+Randle
+Lundy
+Dow
+Chin
+Calvert
+Staton
+Neff
+Kearney
+Darden
+Oakley
+Medeiros
+Mccracken
+Crenshaw
+Block
+Beaver
+Perdue
+Dill
+Whittaker
+Tobin
+Cornelius
+Washburn
+Hogue
+Goodrich
+Easley
+Bravo
+Dennison
+Vera
+Shipley
+Kerns
+Jorgensen
+Crain
+Abel
+Villalobos
+Maurer
+Longoria
+Keene
+Coon
+Sierra
+Witherspoon
+Staples
+Pettit
+Kincaid
+Eason
+Madrid
+Echols
+Lusk
+Wu
+Stahl
+Currie
+Thayer
+Shultz
+Sherwood
+Mcnally
+Seay
+North
+Maher
+Kenny
+Hope
+Gagne
+Barrow
+Nava
+Myles
+Moreland
+Honeycutt
+Hearn
+Diggs
+Caron
+Whitten
+Westbrook
+Stovall
+Ragland
+Queen
+Munson
+Meier
+Looney
+Kimble
+Jolly
+Hobson
+London
+Goddard
+Culver
+Burr
+Presley
+Negron
+Connell
+Tovar
+Marcus
+Huddleston
+Hammer
+Ashby
+Salter
+Root
+Pendleton
+Oleary
+Nickerson
+Myrick
+Judd
+Jacobsen
+Elliot
+Bain
+Adair
+Starnes
+Sheldon
+Matos
+Light
+Busby
+Herndon
+Hanley
+Bellamy
+Jack
+Doty
+Bartley
+Yazzie
+Rowell
+Parson
+Gifford
+Cullen
+Christiansen
+Benavides
+Barnhart
+Talbot
+Mock
+Crandall
+Connors
+Bonds
+Whitt
+Gage
+Bergman
+Arredondo
+Addison
+Marion
+Lujan
+Dowdy
+Jernigan
+Huynh
+Bouchard
+Dutton
+Rhoades
+Ouellette
+Kiser
+Rubin
+Herrington
+Hare
+Denny
+Blackman
+Babb
+Allred
+Rudd
+Paulson
+Ogden
+Koenig
+Jacob
+Irving
+Geiger
+Begay
+Parra
+Champion
+Lassiter
+Hawk
+Esposito
+Cho
+Waldron
+Vernon
+Ransom
+Prather
+Keenan
+Jean
+Grover
+Chacon
+Vick
+Sands
+Roark
+Parr
+Mayberry
+Greenberg
+Coley
+Bruner
+Whitman
+Skaggs
+Shipman
+Means
+Leary
+Hutton
+Romo
+Medrano
+Ladd
+Kruse
+Friend
+Darling
+Askew
+Valentin
+Schulz
+Alfaro
+Tabor
+Mohr
+Gallo
+Bermudez
+Pereira
+Isaac
+Bliss
+Reaves
+Flint
+Comer
+Boston
+Woodall
+Naquin
+Guevara
+Earl
+Delong
+Carrier
+Pickens
+Brand
+Tilley
+Schaffer
+Read
+Lim
+Knutson
+Fenton
+Doran
+Chu
+Vogt
+Vann
+Prescott
+Mclain
+Landis
+Corcoran
+Ambrose
+Zapata
+Hyatt
+Hemphill
+Faulk
+Call
+Dove
+Boudreaux
+Aragon
+Whitlock
+Trejo
+Tackett
+Shearer
+Saldana
+Hanks
+Gold
+Driver
+Mckinnon
+Koehler
+Champagne
+Bourgeois
+Pool
+Keyes
+Goodson
+Foote
+Early
+Lunsford
+Goldsmith
+Flood
+Winslow
+Sams
+Reagan
+Mccloud
+Hough
+Esquivel
+Naylor
+Loomis
+Coronado
+Ludwig
+Braswell
+Bearden
+Sherrill
+Huang
+Fagan
+Ezell
+Edmondson
+Cyr
+Cronin
+Nunn
+Lemon
+Guillory
+Grier
+Dubose
+Traylor
+Ryder
+Dobbins
+Coyle
+Aponte
+Whitmore
+Smalls
+Rowan
+Malloy
+Cardona
+Braxton
+Borden
+Humphries
+Carrasco
+Ruff
+Metzger
+Huntley
+Hinojosa
+Finney
+Madsen
+Hong
+Hills
+Ernst
+Dozier
+Burkhart
+Bowser
+Peralta
+Daigle
+Whittington
+Sorenson
+Saucedo
+Roche
+Redding
+Loyd
+Fugate
+Avalos
+Waite
+Lind
+Huston
+Hay
+Benedict
+Hawthorne
+Hamby
+Boyles
+Boles
+Regan
+Faust
+Crook
+Beam
+Barger
+Hinds
+Gallardo
+Elias
+Willoughby
+Willingham
+Wilburn
+Eckert
+Busch
+Zepeda
+Worthington
+Tinsley
+Russ
+Li
+Hoff
+Hawley
+Carmona
+Varela
+Rector
+Newcomb
+Mallory
+Kinsey
+Dube
+Whatley
+Strange
+Ragsdale
+Ivy
+Bernstein
+Becerra
+Yost
+Mattson
+Ly
+Felder
+Cheek
+Luke
+Handy
+Grossman
+Gauthier
+Escobedo
+Braden
+Beckman
+Mott
+Hillman
+Gil
+Flaherty
+Dykes
+Doe
+Stockton
+Stearns
+Lofton
+Kitchen
+Coats
+Cavazos
+Beavers
+Barrios
+Tang
+Parish
+Mosher
+Lincoln
+Cardwell
+Coles
+Burnham
+Weller
+Lemons
+Beebe
+Aguilera
+Ring
+Parnell
+Harman
+Couture
+Alley
+Schumacher
+Redd
+Dobbs
+Blum
+Blalock
+Merchant
+Ennis
+Denson
+Cottrell
+Chester
+Brannon
+Bagley
+Aviles
+Watt
+Sousa
+Rosenthal
+Rooney
+Dietz
+Blank
+Paquette
+Mcclelland
+Duff
+Velasco
+Lentz
+Grubb
+Burrows
+Barbour
+Ulrich
+Shockley
+Rader
+German
+Beyer
+Mixon
+Layton
+Altman
+Alonzo
+Weathers
+Titus
+Stoner
+Squires
+Shipp
+Priest
+Lipscomb
+Cutler
+Caballero
+Zimmer
+Willett
+Thurston
+Storey
+Medley
+Lyle
+Epperson
+Shah
+Mcmillian
+Baggett
+Torrez
+Laws
+Hirsch
+Dent
+Corey
+Poirier
+Peachey
+Jacques
+Farrar
+Creech
+Barth
+Trimble
+France
+Dupre
+Albrecht
+Sample
+Lawler
+Crisp
+Conroy
+Chadwick
+Wetzel
+Nesbitt
+Murry
+Jameson
+Wilhelm
+Patten
+Minton
+Matson
+Kimbrough
+Iverson
+Guinn
+Gale
+Fortune
+Croft
+Toth
+Pulliam
+Nugent
+Newby
+Littlejohn
+Dias
+Canales
+Bernier
+Baron
+Barney
+Singletary
+Renteria
+Pruett
+Mchugh
+Mabry
+Landrum
+Brower
+Weldon
+Stoddard
+Ruth
+Cagle
+Stjohn
+Scales
+Kohler
+Kellogg
+Hopson
+Gant
+Tharp
+Gann
+Zeigler
+Pringle
+Hammons
+Fairchild
+Deaton
+Chavis
+Carnes
+Rowley
+Matlock
+Libby
+Kearns
+Irizarry
+Carrington
+Starkey
+Pepper
+Lopes
+Jarrell
+Fay
+Craven
+Beverly
+Baum
+Spain
+Littlefield
+Linn
+Humphreys
+Hook
+High
+Etheridge
+Cuellar
+Chastain
+Chance
+Bundy
+Speer
+Skelton
+Quiroz
+Pyle
+Portillo
+Ponder
+Moulton
+Machado
+Liu
+Killian
+Hutson
+Hitchcock
+Ellsworth
+Dowling
+Cloud
+Burdick
+Spann
+Pedersen
+Levin
+Leggett
+Hayward
+Hacker
+Dietrich
+Beaulieu
+Barksdale
+Wakefield
+Snowden
+Paris
+Briscoe
+Bowie
+Berman
+Ogle
+Mcgregor
+Laughlin
+Helm
+Burden
+Wheatley
+Schreiber
+Pressley
+Parris
+Ng
+Alaniz
+Agee
+Urban
+Swann
+Snodgrass
+Schuster
+Radford
+Monk
+Mattingly
+Main
+Lamar
+Harp
+Girard
+Cheney
+Yancey
+Wagoner
+Ridley
+Lombardo
+Lau
+Hudgins
+Gaskins
+Duckworth
+Coe
+Coburn
+Willey
+Prado
+Newberry
+Magana
+Hammonds
+Elam
+Whipple
+Slade
+Serna
+Ojeda
+Liles
+Dorman
+Diehl
+Angel
+Upton
+Reardon
+Michaels
+Kelsey
+Goetz
+Eller
+Bauman
+Baer
+Augustine
+Layne
+Hummel
+Brenner
+Amaya
+Adamson
+Ornelas
+Dowell
+Cloutier
+Christy
+Castellanos
+Wing
+Wellman
+Saylor
+Orourke
+Moya
+Montalvo
+Kilpatrick
+Harley
+Durbin
+Shell
+Oldham
+Kang
+Garvin
+Foss
+Branham
+Bartholomew
+Templeton
+Maguire
+Holton
+Alonso
+Rider
+Monahan
+Mccormack
+Beaty
+Anders
+Streeter
+Nieto
+Nielson
+Moffett
+Lankford
+Keating
+Heck
+Gatlin
+Delatorre
+Callaway
+Adcock
+Worrell
+Unger
+Robinette
+Nowak
+Jeter
+Brunner
+Ashton
+Steen
+Parrott
+Overstreet
+Nobles
+Montanez
+Luther
+Clevenger
+Brinkley
+Trahan
+Quarles
+Pickering
+Pederson
+Jansen
+Grantham
+Gilchrist
+Crespo
+Aiken
+Schell
+Schaeffer
+Lorenz
+Leyva
+Harms
+Dyson
+Wallis
+Pease
+Leavitt
+Hyman
+Cheng
+Cavanaugh
+Batts
+Warden
+Seaman
+Rockwell
+Quezada
+Paxton
+Linder
+Houck
+Fontaine
+Durant
+Caruso
+Adler
+Pimentel
+Mize
+Lytle
+Donald
+Cleary
+Cason
+Acker
+Switzer
+Salmon
+Isaacs
+Higginbotham
+Han
+Waterman
+Vandyke
+Stamper
+Sisk
+Shuler
+Riddick
+Redman
+Mcmahan
+Levesque
+Hatton
+Bronson
+Bollinger
+Arnett
+Okeefe
+Gerber
+Gannon
+Farnsworth
+Baughman
+Silverman
+Satterfield
+Royal
+Mccrary
+Kowalski
+Joy
+Grigsby
+Greco
+Cabral
+Trout
+Rinehart
+Mahon
+Linton
+Gooden
+Curley
+Baugh
+Wyman
+Weiner
+Schwab
+Schuler
+Morrissey
+Mahan
+Coy
+Bunn
+Andrew
+Thrasher
+Spear
+Waggoner
+Shelley
+Robert
+Qualls
+Purdy
+Mcwhorter
+Mauldin
+Mark
+Jordon
+Gilman
+Perryman
+Newsom
+Menard
+Martino
+Graf
+Billingsley
+Artis
+Simpkins
+Salisbury
+Quintanilla
+Gilliland
+Fraley
+Foust
+Crouse
+Scarborough
+Ngo
+Grissom
+Fultz
+Rico
+Marlow
+Markham
+Madrigal
+Lawton
+Barfield
+Whiting
+Varney
+Schwarz
+Huey
+Gooch
+Arce
+Wheat
+Truong
+Poulin
+Mackenzie
+Leone
+Hurtado
+Selby
+Gaither
+Fortner
+Culpepper
+Coughlin
+Brinson
+Boudreau
+Barkley
+Bales
+Stepp
+Holm
+Tan
+Schilling
+Morrell
+Kahn
+Heaton
+Gamez
+Douglass
+Causey
+Brothers
+Turpin
+Shanks
+Schrader
+Meek
+Isom
+Hardison
+Carranza
+Yanez
+Way
+Scroggins
+Schofield
+Runyon
+Ratcliff
+Murrell
+Moeller
+Irby
+Currier
+Butterfield
+Yee
+Ralston
+Pullen
+Pinson
+Estep
+East
+Carbone
+Lance
+Hawks
+Ellington
+Casillas
+Spurlock
+Sikes
+Motley
+Mccartney
+Kruger
+Isbell
+Houle
+Francisco
+Burk
+Bone
+Tomlin
+Shelby
+Quigley
+Neumann
+Lovelace
+Fennell
+Colby
+Cheatham
+Bustamante
+Skidmore
+Hidalgo
+Forman
+Culp
+Bowens
+Betancourt
+Aquino
+Robb
+Rea
+Milner
+Martel
+Gresham
+Wiles
+Ricketts
+Gavin
+Dowd
+Collazo
+Bostic
+Blakely
+Sherrod
+Power
+Kenyon
+Gandy
+Ebert
+Deloach
+Cary
+Bull
+Allard
+Sauer
+Robins
+Olivares
+Gillette
+Chestnut
+Bourque
+Paine
+Lyman
+Hite
+Hauser
+Devore
+Crawley
+Chapa
+Vu
+Tobias
+Talbert
+Poindexter
+Millard
+Meador
+Mcduffie
+Mattox
+Kraus
+Harkins
+Choate
+Bess
+Wren
+Sledge
+Sanborn
+Outlaw
+Kinder
+Geary
+Cornwell
+Barclay
+Adam
+Abney
+Seward
+Rhoads
+Howland
+Fortier
+Easter
+Benner
+Vines
+Tubbs
+Troutman
+Rapp
+Noe
+Mccurdy
+Harder
+Deluca
+Westmoreland
+South
+Havens
+Guajardo
+Ely
+Clary
+Seal
+Meehan
+Herzog
+Guillen
+Ashcraft
+Waugh
+Renner
+Milam
+Jung
+Elrod
+Churchill
+Buford
+Breaux
+Bolin
+Asher
+Windham
+Tirado
+Pemberton
+Nolen
+Noland
+Knott
+Emmons
+Cornish
+Christenson
+Brownlee
+Barbee
+Waldrop
+Pitt
+Olvera
+Lombardi
+Gruber
+Gaffney
+Eggleston
+Banda
+Archuleta
+Still
+Slone
+Prewitt
+Pfeiffer
+Nettles
+Mena
+Mcadams
+Henning
+Gardiner
+Cromwell
+Chisholm
+Burleson
+Box
+Vest
+Oglesby
+Mccarter
+Malcolm
+Lumpkin
+Larue
+Grey
+Wofford
+Vanhorn
+Thorn
+Teel
+Swafford
+Stclair
+Stanfield
+Ocampo
+Herrmann
+Hannon
+Arsenault
+Roush
+Mcalister
+Hiatt
+Gunderson
+Forsythe
+Duggan
+Delvalle
+Cintron
+Wilks
+Weinstein
+Uribe
+Rizzo
+Noyes
+Mclendon
+Gurley
+Bethea
+Winstead
+Maples
+Harry
+Guyton
+Giordano
+Alderman
+Valdes
+Polanco
+Pappas
+Lively
+Grogan
+Griffiths
+Bobo
+Arevalo
+Whitson
+Sowell
+Rendon
+Matthew
+Julian
+Fernandes
+Farrow
+Edmond
+Benavidez
+Ayres
+Alicea
+Stump
+Smalley
+Seitz
+Schulte
+Gilley
+Gallant
+Dewey
+Casper
+Canfield
+Wolford
+Omalley
+Mcnutt
+Mcnulty
+Mcgovern
+Hardman
+Harbin
+Cowart
+Chavarria
+Brink
+Beckett
+Bagwell
+Armstead
+Anglin
+Abreu
+Reynoso
+Krebs
+Jett
+Hoffmann
+Greenfield
+Forte
+Burney
+Broome
+Sisson
+Parent
+Jude
+Younger
+Trammell
+Partridge
+Marvin
+Mace
+Lomax
+Lemieux
+Gossett
+Frantz
+Fogle
+Cooney
+Broughton
+Pence
+Paulsen
+Neil
+Muncy
+Mcarthur
+Hollins
+Edward
+Beauchamp
+Withers
+Osorio
+Mulligan
+Hoyle
+Foy
+Dockery
+Cockrell
+Begley
+Amador
+Roby
+Rains
+Lindquist
+Gentile
+Everhart
+Bohannon
+Wylie
+Thao
+Sommers
+Purnell
+Palma
+Fortin
+Dunning
+Breeden
+Vail
+Phelan
+Phan
+Marx
+Cosby
+Colburn
+Chong
+Boling
+Biddle
+Ledesma
+Gaddis
+Denney
+Chow
+Bueno
+Berrios
+Wicker
+Tolliver
+Thibodeaux
+Nagle
+Lavoie
+Fisk
+Do
+Crist
+Barbosa
+Reedy
+March
+Locklear
+Kolb
+Himes
+Behrens
+Beckwith
+Beckham
+Weems
+Wahl
+Shorter
+Shackelford
+Rees
+Muse
+Free
+Cerda
+Valadez
+Thibodeau
+Saavedra
+Ridgeway
+Reiter
+Mchenry
+Majors
+Lachance
+Keaton
+Israel
+Ferrara
+Falcon
+Clemens
+Blocker
+Applegate
+Paz
+Needham
+Mojica
+Kuykendall
+Hamel
+Escamilla
+Doughty
+Burchett
+Ainsworth
+Wilbur
+Vidal
+Upchurch
+Thigpen
+Strauss
+Spruill
+Sowers
+Riggins
+Ricker
+Mccombs
+Harlow
+Garnett
+Buffington
+Yi
+Sotelo
+Olivas
+Negrete
+Morey
+Macon
+Logsdon
+Lapointe
+Florence
+Cathey
+Bigelow
+Bello
+Westfall
+Stubblefield
+Peak
+Lindley
+Jeffrey
+Hein
+Hawes
+Farrington
+Edge
+Breen
+Birch
+Wilde
+Steed
+Sepulveda
+Reinhardt
+Proffitt
+Minter
+Messina
+Mcnabb
+Maier
+Keeler
+Gamboa
+Donohue
+Dexter
+Basham
+Shinn
+Orlando
+Crooks
+Cota
+Borders
+Bills
+Bachman
+Tisdale
+Tavares
+Schmid
+Pickard
+Jasper
+Gulley
+Fonseca
+Delossantos
+Condon
+Clancy
+Batista
+Wicks
+Wadsworth
+New
+Martell
+Lo
+Littleton
+Ison
+Haag
+Folsom
+Brumfield
+Broyles
+Brito
+Mireles
+Mcdonnell
+Leclair
+Hamblin
+Gough
+Fanning
+Binder
+Winfield
+Whitworth
+Soriano
+Palumbo
+Newkirk
+Mangum
+Hutcherson
+Comstock
+Cecil
+Carlin
+Beall
+Bair
+Wendt
+Watters
+Walling
+Putman
+Otoole
+Oliva
+Morley
+Mares
+Lemus
+Keener
+Jeffery
+Hundley
+Dial
+Damico
+Billups
+Strother
+Mcfarlane
+Lamm
+Eaves
+Crutcher
+Caraballo
+Canty
+Atwell
+Taft
+Siler
+Rust
+Rawls
+Rawlings
+Prieto
+Niles
+Mcneely
+Mcafee
+Hulsey
+Harlan
+Hackney
+Galvez
+Escalante
+Delagarza
+Crider
+Charlton
+Bandy
+Wilbanks
+Stowe
+Steinberg
+Samson
+Renfro
+Masterson
+Massie
+Lanham
+Haskell
+Hamrick
+Fort
+Dehart
+Card
+Burdette
+Branson
+Bourne
+Babin
+Aleman
+Worthy
+Tibbs
+Sweat
+Smoot
+Slack
+Paradis
+Packard
+Mull
+Luce
+Houghton
+Gantt
+Furman
+Danner
+Christianson
+Burge
+Broderick
+Ashford
+Arndt
+Almeida
+Stallworth
+Shade
+Searcy
+Sager
+Noonan
+Mclemore
+Mcintire
+Maxey
+Lavigne
+Jobe
+Ireland
+Ferrer
+Falk
+Edgar
+Coffin
+Byrnes
+Aranda
+Apodaca
+Stamps
+Rounds
+Peek
+Olmstead
+Lewandowski
+Kaminski
+Her
+Dunaway
+Bruns
+Brackett
+Amato
+Reich
+Mcclung
+Lacroix
+Koontz
+Herrick
+Hardesty
+Flanders
+Cousins
+Close
+Cato
+Cade
+Vickery
+Shank
+Nagel
+Dupuis
+Croteau
+Cotter
+Cable
+Stuckey
+Stine
+Porterfield
+Pauley
+Nye
+Moffitt
+Lu
+Knudsen
+Hardwick
+Goforth
+Dupont
+Blunt
+Barrows
+Barnhill
+Shull
+Rash
+Ralph
+Penny
+Lorenzo
+Loftis
+Lemay
+Kitchens
+Horvath
+Grenier
+Fuchs
+Fairbanks
+Culbertson
+Calkins
+Burnside
+Beattie
+Ashworth
+Albertson
+Wertz
+Vo
+Vaught
+Vallejo
+Tyree
+Turk
+Tuck
+Tijerina
+Sage
+Picard
+Peterman
+Otis
+Marroquin
+Marr
+Lantz
+Hoang
+Demarco
+Daily
+Cone
+Berube
+Barnette
+Wharton
+Stinnett
+Slocum
+Scanlon
+Sander
+Pinto
+Mancuso
+Lima
+Judge
+Headley
+Epstein
+Counts
+Clarkson
+Carnahan
+Brice
+Boren
+Arteaga
+Adame
+Zook
+Whittle
+Whitehurst
+Wenzel
+Saxton
+Rhea
+Reddick
+Puente
+Hazel
+Handley
+Haggerty
+Earley
+Devlin
+Dallas
+Chaffin
+Cady
+Ahmed
+Acuna
+Solano
+Sigler
+Pollack
+Pendergrass
+Ostrander
+Janes
+Francois
+Fine
+Crutchfield
+Cordell
+Chamberlin
+Brubaker
+Baptiste
+Willson
+Reis
+Neeley
+Mullin
+Mercier
+Lira
+Layman
+Keeling
+Higdon
+Guest
+Forrester
+Espinal
+Dion
+Chapin
+Carl
+Warfield
+Toledo
+Pulido
+Peebles
+Nagy
+Montague
+Mello
+Lear
+Jaeger
+Hogg
+Graff
+Furr
+Derrick
+Cave
+Canada
+Soliz
+Poore
+Mendenhall
+Mclaurin
+Maestas
+Low
+Gable
+Belt
+Barraza
+Tillery
+Snead
+Pond
+Neill
+Mcculloch
+Mccorkle
+Lightfoot
+Hutchings
+Holloman
+Harness
+Dorn
+Council
+Bock
+Zielinski
+Turley
+Treadwell
+Stpierre
+Starling
+Somers
+Oswald
+Merrick
+Marquis
+Ivory
+Easterling
+Bivens
+Truitt
+Poston
+Parry
+Ontiveros
+Olivarez
+Neville
+Moreau
+Medlin
+Ma
+Lenz
+Knowlton
+Fairley
+Cobbs
+Chisolm
+Bannister
+Woodworth
+Toler
+Ocasio
+Noriega
+Neuman
+Moye
+Milburn
+Mcclanahan
+Lilley
+Hanes
+Flannery
+Dellinger
+Danielson
+Conti
+Blodgett
+Beers
+Weatherford
+Strain
+Karr
+Hitt
+Denham
+Custer
+Coble
+Clough
+Casteel
+Bolduc
+Batchelor
+Ammons
+Whitlow
+Tierney
+Staten
+Sibley
+Seifert
+Schubert
+Salcedo
+Mattison
+Laney
+Haggard
+Grooms
+Dix
+Dees
+Cromer
+Cooks
+Colson
+Caswell
+Zarate
+Swisher
+Stacey
+Shin
+Ragan
+Pridgen
+Mcvey
+Matheny
+Leigh
+Lafleur
+Franz
+Ferraro
+Dugger
+Whiteside
+Rigsby
+Mcmurray
+Lehmann
+Large
+Jacoby
+Hildebrand
+Hendrick
+Headrick
+Goad
+Fincher
+Drury
+Borges
+Archibald
+Albers
+Woodcock
+Trapp
+Soares
+Seaton
+Richie
+Monson
+Luckett
+Lindberg
+Kopp
+Keeton
+Hsu
+Healey
+Garvey
+Gaddy
+Fain
+Burchfield
+Badger
+Wentworth
+Strand
+Stack
+Spooner
+Saucier
+Sales
+Ruby
+Ricci
+Plunkett
+Pannell
+Ness
+Leger
+Hoy
+Freitas
+Fong
+Elizondo
+Duval
+Chun
+Calvin
+Beaudoin
+Urbina
+Stock
+Rickard
+Partin
+Moe
+Mcgrew
+Mcclintock
+Ledoux
+Forsyth
+Faison
+Devries
+Bertrand
+Wasson
+Tilton
+Scarbrough
+Pride
+Oh
+Leung
+Larry
+Irvine
+Garber
+Denning
+Corral
+Colley
+Castleberry
+Bowlin
+Bogan
+Beale
+Baines
+True
+Trice
+Rayburn
+Parkinson
+Pak
+Nunes
+Mcmillen
+Leahy
+Lea
+Kimmel
+Higgs
+Fulmer
+Carden
+Bedford
+Taggart
+Spearman
+Register
+Prichard
+Morrill
+Koonce
+Heinz
+Hedges
+Guenther
+Grice
+Findley
+Earle
+Dover
+Creighton
+Boothe
+Bayer
+Arreola
+Vitale
+Valles
+See
+Raney
+Peter
+Osgood
+Lowell
+Hanlon
+Burley
+Bounds
+Worden
+Weatherly
+Vetter
+Tanaka
+Stiltner
+Sell
+Nevarez
+Mosby
+Montero
+Melancon
+Harter
+Hamer
+Goble
+Gladden
+Gist
+Ginn
+Akin
+Zaragoza
+Towns
+Tarver
+Sammons
+Royster
+Oreilly
+Muir
+Morehead
+Luster
+Kingsley
+Kelso
+Grisham
+Glynn
+Baumann
+Alves
+Yount
+Tamayo
+Tam
+Paterson
+Oates
+Menendez
+Longo
+Hargis
+Greenlee
+Gillen
+Desantis
+Conover
+Breedlove
+Wayne
+Sumpter
+Scherer
+Rupp
+Reichert
+Heredia
+Fallon
+Creel
+Cohn
+Clemmons
+Casas
+Bickford
+Belton
+Bach
+Williford
+Whitcomb
+Tennant
+Sutter
+Stull
+Sessions
+Mccallum
+Manson
+Langlois
+Keel
+Keegan
+Emanuel
+Dangelo
+Dancy
+Damron
+Clapp
+Clanton
+Bankston
+Trinidad
+Oliveira
+Mintz
+Mcinnis
+Martens
+Mabe
+Laster
+Jolley
+Irish
+Hildreth
+Hefner
+Glaser
+Duckett
+Demers
+Brockman
+Blais
+Back
+Alcorn
+Agnew
+Toliver
+Tice
+Song
+Seeley
+Najera
+Musser
+Mcfall
+Laplante
+Galvin
+Fajardo
+Doan
+Coyne
+Copley
+Clawson
+Cheung
+Barone
+Wynne
+Woodley
+Tremblay
+Stoll
+Sparrow
+Sparkman
+Schweitzer
+Sasser
+Samples
+Roney
+Ramon
+Legg
+Lai
+Joe
+Heim
+Farias
+Concepcion
+Colwell
+Christman
+Bratcher
+Alba
+Winchester
+Upshaw
+Southerland
+Sorrell
+Shay
+Sells
+Mount
+Mccloskey
+Martindale
+Luttrell
+Loveless
+Lovejoy
+Linares
+Latimer
+Holly
+Embry
+Coombs
+Bratton
+Bostick
+Boss
+Venable
+Tuggle
+Toro
+Staggs
+Sandlin
+Jefferies
+Heckman
+Griffis
+Crayton
+Clem
+Button
+Browder
+Allan
+Thorton
+Sturgill
+Sprouse
+Royer
+Rousseau
+Ridenour
+Pogue
+Perales
+Peeples
+Metzler
+Mesa
+Mccutcheon
+Mcbee
+Jay
+Hornsby
+Heffner
+Corrigan
+Armijo
+Vue
+Romeo
+Plante
+Peyton
+Paredes
+Macklin
+Hussey
+Hodgson
+Granados
+Frias
+Carman
+Brent
+Becnel
+Batten
+Almanza
+Turney
+Teal
+Sturgeon
+Meeker
+Mcdaniels
+Limon
+Keeney
+Kee
+Hutto
+Holguin
+Gorham
+Fishman
+Fierro
+Blanchette
+Rodrigue
+Reddy
+Osburn
+Oden
+Lerma
+Kirkwood
+Keefer
+Haugen
+Hammett
+Chalmers
+Carlos
+Brinkman
+Baumgartner
+Zhang
+Valerio
+Tellez
+Steffen
+Shumate
+Sauls
+Ripley
+Kemper
+Jacks
+Guffey
+Evers
+Craddock
+Carvalho
+Blaylock
+Banuelos
+Balderas
+Wooden
+Wheaton
+Turnbull
+Shuman
+Pointer
+Mosier
+Mccue
+Ligon
+Kozlowski
+Johansen
+Ingle
+Herr
+Briones
+Southern
+Snipes
+Rickman
+Pipkin
+Peace
+Pantoja
+Orosco
+Moniz
+Lawless
+Kunkel
+Hibbard
+Galarza
+Enos
+Bussey
+Settle
+Schott
+Salcido
+Perreault
+Mcdougal
+Mccool
+Haight
+Garris
+Ferry
+Easton
+Conyers
+Atherton
+Wimberly
+Utley
+Stephen
+Spellman
+Smithson
+Slagle
+Skipper
+Ritchey
+Rand
+Petit
+Osullivan
+Oaks
+Nutt
+Mcvay
+Mccreary
+Mayhew
+Knoll
+Jewett
+Harwood
+Hailey
+Cardoza
+Ashe
+Arriaga
+Andres
+Zeller
+Wirth
+Whitmire
+Stauffer
+Spring
+Rountree
+Redden
+Mccaffrey
+Martz
+Loving
+Larose
+Langdon
+Humes
+Gaskin
+Faber
+Doll
+Devito
+Cass
+Almond
+Wingfield
+Wingate
+Villareal
+Tyner
+Smothers
+Severson
+Reno
+Pennell
+Maupin
+Leighton
+Janssen
+Hassell
+Hallman
+Halcomb
+Folse
+Fitzsimmons
+Fahey
+Cranford
+Bolen
+Battles
+Battaglia
+Wooldridge
+Weed
+Trask
+Rosser
+Regalado
+Mcewen
+Keefe
+Fuqua
+Echevarria
+Domingo
+Dang
+Caro
+Boynton
+Andrus
+Wild
+Viera
+Vanmeter
+Taber
+Spradlin
+Seibert
+Provost
+Prentice
+Oliphant
+Laporte
+Hwang
+Hatchett
+Hass
+Greiner
+Freedman
+Covert
+Chilton
+Byars
+Wiese
+Venegas
+Swank
+Shrader
+Roderick
+Roberge
+Mullis
+Mortensen
+Mccune
+Marlowe
+Kirchner
+Keck
+Isaacson
+Hostetler
+Halverson
+Gunther
+Griswold
+Gerard
+Fenner
+Durden
+Blackwood
+Bertram
+Ahrens
+Sawyers
+Savoy
+Nabors
+Mcswain
+Mackay
+Loy
+Lavender
+Lash
+Labbe
+Jessup
+Hubert
+Fullerton
+Donnell
+Cruse
+Crittenden
+Correia
+Centeno
+Caudle
+Canady
+Callender
+Alarcon
+Ahern
+Winfrey
+Tribble
+Tom
+Styles
+Salley
+Roden
+Musgrove
+Minnick
+Fortenberry
+Carrion
+Bunting
+Bethel
+Batiste
+Woo
+Whited
+Underhill
+Stillwell
+Silvia
+Rauch
+Pippin
+Perrin
+Messenger
+Mancini
+Lister
+Kinard
+Hartmann
+Fleck
+Broadway
+Wilt
+Treadway
+Thornhill
+Speed
+Spalding
+Sam
+Rafferty
+Pitre
+Patino
+Ordonez
+Linkous
+Kelleher
+Homan
+Holiday
+Galbraith
+Feeney
+Dorris
+Curtin
+Coward
+Camarillo
+Buss
+Bunnell
+Bolt
+Beeler
+Autry
+Alcala
+Witte
+Wentz
+Stidham
+Shively
+Nunley
+Meacham
+Martins
+Lemke
+Lefebvre
+Kaye
+Hynes
+Horowitz
+Hoppe
+Holcombe
+Estrella
+Dunne
+Derr
+Cochrane
+Brittain
+Bedard
+Beauregard
+Torrence
+Strunk
+Soria
+Simonson
+Shumaker
+Scoggins
+Packer
+Oconner
+Moriarty
+Leroy
+Kuntz
+Ives
+Hutcheson
+Horan
+Hales
+Garmon
+Fitts
+Dell
+Bohn
+Atchison
+Worth
+Wisniewski
+Will
+Vanwinkle
+Sturm
+Sallee
+Prosser
+Moen
+Lundberg
+Kunz
+Kohl
+Keane
+Jorgenson
+Jaynes
+Funderburk
+Freed
+Frame
+Durr
+Creamer
+Cosgrove
+Candelaria
+Berlin
+Batson
+Vanhoose
+Thomsen
+Teeter
+Sommer
+Smyth
+Sena
+Redmon
+Orellana
+Maness
+Lennon
+Heflin
+Goulet
+Frick
+Forney
+Dollar
+Bunker
+Asbury
+Aguiar
+Talbott
+Southard
+Pleasant
+Mowery
+Mears
+Lemmon
+Krieger
+Hickson
+Gracia
+Elston
+Duong
+Delgadillo
+Dayton
+Dasilva
+Conaway
+Catron
+Bruton
+Bradbury
+Bordelon
+Bivins
+Bittner
+Bergstrom
+Beals
+Abell
+Whelan
+Travers
+Tejada
+Pulley
+Pino
+Norfleet
+Nealy
+Maes
+Loper
+Held
+Gerald
+Gatewood
+Frierson
+Freund
+Finnegan
+Cupp
+Covey
+Catalano
+Boehm
+Bader
+Yoon
+Walston
+Tenney
+Sipes
+Roller
+Rawlins
+Medlock
+Mccaskill
+Mccallister
+Marcotte
+Maclean
+Hughey
+Henke
+Harwell
+Gladney
+Gilson
+Dew
+Chism
+Caskey
+Brandenburg
+Baylor
+Villasenor
+Veal
+Van
+Thatcher
+Stegall
+Shore
+Petrie
+Nowlin
+Navarrete
+Muhammad
+Lombard
+Loftin
+Lemaster
+Kroll
+Kovach
+Kimbrell
+Kidwell
+Hershberger
+Fulcher
+Eng
+Cantwell
+Bustos
+Boland
+Bobbitt
+Binkley
+Wester
+Weis
+Verdin
+Tong
+Tiller
+Sisco
+Sharkey
+Seymore
+Rosenbaum
+Rohr
+Quinonez
+Pinkston
+Nation
+Malley
+Logue
+Lessard
+Lerner
+Lebron
+Krauss
+Klinger
+Halstead
+Haller
+Getz
+Burrow
+Brant
+Alger
+Victor
+Shores
+Scully
+Pounds
+Pfeifer
+Perron
+Nelms
+Munn
+Mcmaster
+Mckenney
+Manns
+Knudson
+Hutchens
+Huskey
+Goebel
+Flagg
+Cushman
+Click
+Castellano
+Carder
+Bumgarner
+Blaine
+Bible
+Wampler
+Spinks
+Robson
+Neel
+Mcreynolds
+Mathias
+Maas
+Loera
+Kasper
+Jose
+Jenson
+Florez
+Coons
+Buckingham
+Brogan
+Berryman
+Wilmoth
+Wilhite
+Thrash
+Shephard
+Seidel
+Schulze
+Roldan
+Pettis
+Obryan
+Maki
+Mackie
+Hatley
+Frazer
+Fiore
+Falls
+Chesser
+Bui
+Bottoms
+Bisson
+Benefield
+Allman
+Wilke
+Trudeau
+Timm
+Shifflett
+Rau
+Mundy
+Milliken
+Mayers
+Leake
+Kohn
+Huntington
+Horsley
+Hermann
+Guerin
+Fryer
+Frizzell
+Foret
+Flemming
+Fife
+Criswell
+Carbajal
+Bozeman
+Boisvert
+Archie
+Antonio
+Angulo
+Wallen
+Tapp
+Silvers
+Ramsay
+Oshea
+Orta
+Moll
+Mckeever
+Mcgehee
+Luciano
+Linville
+Kiefer
+Ketchum
+Howerton
+Groce
+Gaylord
+Gass
+Fusco
+Corbitt
+Blythe
+Betz
+Bartels
+Amaral
+Aiello
+Yoo
+Weddle
+Troy
+Sun
+Sperry
+Seiler
+Runyan
+Raley
+Overby
+Osteen
+Olds
+Mckeown
+Mauro
+Matney
+Lauer
+Lattimore
+Hindman
+Hartwell
+Fredrickson
+Fredericks
+Espino
+Clegg
+Carswell
+Cambell
+Burkholder
+August
+Woodbury
+Welker
+Totten
+Thornburg
+Theriault
+Stitt
+Stamm
+Stackhouse
+Simone
+Scholl
+Saxon
+Rife
+Razo
+Quinlan
+Pinkerton
+Olivo
+Nesmith
+Nall
+Mattos
+Leak
+Lafferty
+Justus
+Giron
+Geer
+Fielder
+Eagle
+Drayton
+Dortch
+Conners
+Conger
+Chau
+Boatwright
+Billiot
+Barden
+Armenta
+Antoine
+Tibbetts
+Steadman
+Slattery
+Sides
+Rinaldi
+Raynor
+Rayford
+Pinckney
+Pettigrew
+Nickel
+Milne
+Matteson
+Halsey
+Gonsalves
+Fellows
+Durand
+Desimone
+Cowley
+Cowles
+Brill
+Barham
+Barela
+Barba
+Ashmore
+Withrow
+Valenti
+Tejeda
+Spriggs
+Sayre
+Salerno
+Place
+Peltier
+Peel
+Merriman
+Matheson
+Lowman
+Lindstrom
+Hyland
+Homer
+Ha
+Giroux
+Fries
+Frasier
+Earls
+Dugas
+Damon
+Dabney
+Collado
+Briseno
+Baxley
+Andre
+Word
+Whyte
+Wenger
+Vanover
+Vanburen
+Thiel
+Schindler
+Schiller
+Rigby
+Pomeroy
+Passmore
+Marble
+Manzo
+Mahaffey
+Lindgren
+Laflamme
+Greathouse
+Fite
+Ferrari
+Calabrese
+Bayne
+Yamamoto
+Wick
+Townes
+Thames
+Steel
+Reinhart
+Peeler
+Naranjo
+Montez
+Mcdade
+Mast
+Markley
+Marchand
+Leeper
+Kong
+Kellum
+Hudgens
+Hennessey
+Hadden
+Guess
+Gainey
+Coppola
+Borrego
+Bolling
+Beane
+Ault
+Slaton
+Poland
+Pape
+Null
+Mulkey
+Lightner
+Langer
+Hillard
+Glasgow
+Fabian
+Ethridge
+Enright
+Derosa
+Baskin
+Alfred
+Weinberg
+Turman
+Tinker
+Somerville
+Pardo
+Noll
+Lashley
+Ingraham
+Hiller
+Hendon
+Glaze
+Flora
+Cothran
+Cooksey
+Conte
+Carrico
+Apple
+Abner
+Wooley
+Swope
+Summerlin
+Sturgis
+Sturdivant
+Stott
+Spurgeon
+Spillman
+Speight
+Roussel
+Popp
+Nutter
+Mckeon
+Mazza
+Magnuson
+Lanning
+Kozak
+Jankowski
+Heyward
+Forster
+Corwin
+Callaghan
+Bays
+Wortham
+Usher
+Theriot
+Sayers
+Sabo
+Rupert
+Poling
+Nathan
+Loya
+Lieberman
+Levi
+Laroche
+Labelle
+Howes
+Harr
+Garay
+Fogarty
+Everson
+Durkin
+Dominquez
+Chaves
+Chambliss
+Alfonso
+Witcher
+Wilber
+Vieira
+Vandiver
+Terrill
+Stoker
+Schreiner
+Nestor
+Moorman
+Liddell
+Lew
+Lawhorn
+Krug
+Irons
+Hylton
+Hollenbeck
+Herrin
+Hembree
+Hair
+Goolsby
+Goodin
+Gilmer
+Foltz
+Dinkins
+Daughtry
+Caban
+Brim
+Briley
+Bilodeau
+Bear
+Wyant
+Vergara
+Tallent
+Swearingen
+Stroup
+Sherry
+Scribner
+Roger
+Quillen
+Pitman
+Monaco
+Mccants
+Maxfield
+Martinson
+Landon
+Holtz
+Flournoy
+Brookins
+Brody
+Baumgardner
+Angelo
+Straub
+Sills
+Roybal
+Roundtree
+Oswalt
+Money
+Mcgriff
+Mcdougall
+Mccleary
+Maggard
+Gragg
+Gooding
+Godinez
+Doolittle
+Donato
+Cowell
+Cassell
+Bracken
+Appel
+Ahmad
+Zambrano
+Reuter
+Perea
+Olive
+Nakamura
+Monaghan
+Mickens
+Mcclinton
+Mcclary
+Marler
+Kish
+Judkins
+Gilbreath
+Freese
+Flanigan
+Felts
+Erdmann
+Dodds
+Chew
+Brownell
+Brazil
+Boatright
+Barreto
+Slayton
+Sandberg
+Saldivar
+Pettway
+Odum
+Narvaez
+Moultrie
+Montemayor
+Merrell
+Lees
+Keyser
+Hoke
+Hardaway
+Hannan
+Gilbertson
+Fogg
+Dumont
+Deberry
+Coggins
+Carrera
+Buxton
+Bucher
+Broadnax
+Beeson
+Araujo
+Appleton
+Amundson
+Aguayo
+Ackley
+Yocum
+Worsham
+Shivers
+Shelly
+Sanches
+Sacco
+Robey
+Rhoden
+Pender
+Ochs
+Mccurry
+Madera
+Luong
+Luis
+Knotts
+Jackman
+Heinrich
+Hargrave
+Gault
+Forest
+Comeaux
+Chitwood
+Child
+Caraway
+Boettcher
+Bernhardt
+Barrientos
+Zink
+Wickham
+Whiteman
+Thorp
+Stillman
+Settles
+Schoonover
+Roque
+Riddell
+Rey
+Pilcher
+Phifer
+Novotny
+Maple
+Macleod
+Hardee
+Haase
+Grider
+Fredrick
+Earnest
+Doucette
+Clausen
+Christmas
+Bevins
+Beamon
+Badillo
+Tolley
+Tindall
+Soule
+Snook
+Sebastian
+Seale
+Pitcher
+Pinkney
+Pellegrino
+Nowell
+Nemeth
+Nail
+Mondragon
+Mclane
+Lundgren
+Ingalls
+Hudspeth
+Hixson
+Gearhart
+Furlong
+Downes
+Dionne
+Dibble
+Deyoung
+Cornejo
+Camara
+Brookshire
+Boyette
+Wolcott
+Tracey
+Surratt
+Sellars
+Segal
+Salyer
+Reeve
+Rausch
+Philips
+Labonte
+Haro
+Gower
+Freeland
+Fawcett
+Eads
+Driggers
+Donley
+Collett
+Cage
+Bromley
+Boatman
+Ballinger
+Baldridge
+Volz
+Trombley
+Stonge
+Silas
+Shanahan
+Rivard
+Rhyne
+Pedroza
+Matias
+Mallard
+Jamieson
+Hedgepeth
+Hartnett
+Estevez
+Eskridge
+Denman
+Chiu
+Chinn
+Catlett
+Carmack
+Buie
+Book
+Bechtel
+Beardsley
+Bard
+Ballou
+Windsor
+Ulmer
+Storm
+Skeen
+Robledo
+Rincon
+Reitz
+Piazza
+Pearl
+Munger
+Moten
+Mcmichael
+Loftus
+Ledet
+Kersey
+Groff
+Fowlkes
+Folk
+Crumpton
+Collette
+Clouse
+Bettis
+Villagomez
+Timmerman
+Strom
+Saul
+Santoro
+Roddy
+Phillip
+Penrod
+Musselman
+Macpherson
+Leboeuf
+Harless
+Haddad
+Guido
+Golding
+Fulkerson
+Fannin
+Dulaney
+Dowdell
+Deane
+Cottle
+Ceja
+Cate
+Bosley
+Benge
+Albritton
+Voigt
+Trowbridge
+Soileau
+Seely
+Rome
+Rohde
+Pearsall
+Paulk
+Orth
+Nason
+Mota
+Mcmullin
+Marquardt
+Madigan
+Hoag
+Gillum
+Gayle
+Gabbard
+Fenwick
+Fender
+Eck
+Danforth
+Cushing
+Cress
+Creed
+Cazares
+Casanova
+Bey
+Bettencourt
+Barringer
+Baber
+Stansberry
+Schramm
+Rutter
+Rivero
+Race
+Oquendo
+Necaise
+Mouton
+Montenegro
+Miley
+Mcgough
+Marra
+Macmillan
+Lock
+Lamontagne
+Jasso
+Jaime
+Horst
+Hetrick
+Heilman
+Gaytan
+Gall
+Fried
+Fortney
+Eden
+Dingle
+Desjardins
+Dabbs
+Burbank
+Brigham
+Breland
+Beaman
+Banner
+Arriola
+Yarborough
+Wallin
+Treat
+Toscano
+Stowers
+Reiss
+Pichardo
+Orton
+Mitchel
+Michels
+Mcnamee
+Mccrory
+Leatherman
+Kell
+Keister
+Jerome
+Horning
+Hargett
+Guay
+Friday
+Ferro
+Deboer
+Dagostino
+Clemente
+Christ
+Carper
+Bowler
+Blanks
+Beaudry
+Willie
+Towle
+Tafoya
+Stricklin
+Strader
+Soper
+Sonnier
+Sigmon
+Schenk
+Saddler
+Rodman
+Pedigo
+Mendes
+Lunn
+Lohr
+Lahr
+Kingsbury
+Jarman
+Hume
+Holliman
+Hofmann
+Haworth
+Harrelson
+Hambrick
+Flick
+Edmunds
+Dacosta
+Crossman
+Colston
+Chaplin
+Carrell
+Budd
+Weiler
+Waits
+Viola
+Valentino
+Trantham
+Tarr
+Straight
+Solorio
+Roebuck
+Powe
+Plank
+Pettus
+Palm
+Pagano
+Mink
+Luker
+Leathers
+Joslin
+Hartzell
+Gambrell
+Fears
+Deutsch
+Cepeda
+Carty
+Caputo
+Brewington
+Bedell
+Ballew
+Applewhite
+Warnock
+Walz
+Urena
+Tudor
+Reel
+Pigg
+Parton
+Mickelson
+Meagher
+Mclellan
+Mcculley
+Mandel
+Leech
+Lavallee
+Kraemer
+Kling
+Kipp
+Kingston
+Kehoe
+Hochstetler
+Harriman
+Gregoire
+Grabowski
+Gosselin
+Gammon
+Fancher
+Edens
+Desai
+Butt
+Brannan
+Armendariz
+Woolsey
+Whitehouse
+Whetstone
+Ussery
+Towne
+Tower
+Testa
+Tallman
+Studer
+Strait
+Steinmetz
+Sorrells
+Sauceda
+Rolfe
+Rae
+Paddock
+Mitchem
+Mcginn
+Mccrea
+Luck
+Lovato
+Ling
+Hazen
+Gilpin
+Gaynor
+Fike
+Devoe
+Delrio
+Curiel
+Burkhardt
+Bristol
+Bode
+Backus
+Alton
+Zinn
+Watanabe
+Wachter
+Vanpelt
+Turnage
+Shaner
+Schroder
+Sato
+Riordan
+Quimby
+Portis
+Natale
+Mckoy
+Mccown
+Marker
+Lucio
+Kilmer
+Karl
+Hotchkiss
+Hesse
+Halbert
+Gwinn
+Godsey
+Desmond
+Delisle
+Chrisman
+Canter
+Brook
+Arbogast
+Angell
+Acree
+Yancy
+Woolley
+Wesson
+Weatherspoon
+Trainor
+Stockman
+Spiller
+Sipe
+Rooks
+Reavis
+Propst
+Porras
+Neilson
+Mullens
+Loucks
+Llewellyn
+Lamont
+Kumar
+Koester
+Klingensmith
+Kirsch
+Kester
+Honaker
+Hodson
+Hennessy
+Helmick
+Garrity
+Garibay
+Fee
+Drain
+Casarez
+Callis
+Botello
+Bay
+Aycock
+Avant
+Angle
+Wingard
+Wayman
+Tully
+Theisen
+Szymanski
+Stansbury
+Segovia
+Rudy
+Rainwater
+Preece
+Pirtle
+Padron
+Mincey
+Mckelvey
+Mathes
+Marty
+Larrabee
+Kornegay
+Klug
+Judy
+Ingersoll
+Hecht
+Germain
+Eggers
+Dykstra
+Denis
+Deering
+Decoteau
+Deason
+Dearing
+Cofield
+Carrigan
+Brush
+Bonham
+Bahr
+Aucoin
+Appleby
+Almonte
+Yager
+Womble
+Wimmer
+Weimer
+Vanderpool
+Stancil
+Sprinkle
+Romine
+Remington
+Pfaff
+Peckham
+Olivera
+Meraz
+Maze
+Lathrop
+Koehn
+Jonas
+Hazelton
+Halvorson
+Hallock
+Haddock
+Ducharme
+Dehaven
+Colton
+Caruthers
+Brehm
+Bosworth
+Bost
+Blow
+Bias
+Beeman
+Basile
+Bane
+Aikens
+Zachary
+Wold
+Walther
+Tabb
+Suber
+Strawn
+Stocks
+Stocker
+Shirey
+Schlosser
+Salvador
+Riedel
+Rembert
+Reimer
+Pyles
+Pickle
+Peele
+Merriweather
+Letourneau
+Latta
+Kidder
+Hixon
+Hillis
+Hight
+Herbst
+Henriquez
+Haygood
+Hamill
+Gabel
+Fritts
+Eubank
+Duty
+Dawes
+Correll
+Coffee
+Cha
+Bushey
+Buchholz
+Brotherton
+Bridge
+Botts
+Barnwell
+Auger
+Atchley
+Westphal
+Veilleux
+Ulloa
+Truman
+Stutzman
+Shriver
+Ryals
+Prior
+Pilkington
+Newport
+Moyers
+Miracle
+Marrs
+Mangrum
+Maddux
+Lockard
+Laing
+Kuhl
+Harney
+Hammock
+Hamlett
+Felker
+Doerr
+Depriest
+Carrasquillo
+Carothers
+Bogle
+Blood
+Bischoff
+Bergen
+Albanese
+Wyckoff
+Vermillion
+Vansickle
+Thibault
+Tetreault
+Stickney
+Shoemake
+Ruggiero
+Rawson
+Racine
+Philpot
+Paschal
+Mcelhaney
+Mathison
+Legrand
+Lapierre
+Kwan
+Kremer
+Jiles
+Hilbert
+Geyer
+Faircloth
+Ehlers
+Egbert
+Desrosiers
+Dalrymple
+Cotten
+Cashman
+Cadena
+Breeding
+Boardman
+Alcaraz
+Ahn
+Wyrick
+Therrien
+Tankersley
+Strickler
+Puryear
+Plourde
+Pattison
+Pardue
+Milan
+Mcginty
+Mcevoy
+Landreth
+Kuhns
+Koon
+Hewett
+Giddens
+Everette
+Emerick
+Eades
+Deangelis
+Cosme
+Ceballos
+Birdsong
+Benham
+Bemis
+Armour
+Anguiano
+Angeles
+Welborn
+Tsosie
+Storms
+Shoup
+Sessoms
+Samaniego
+Rood
+Rojo
+Rhinehart
+Raby
+Northcutt
+Myer
+Munguia
+Morehouse
+More
+Mcdevitt
+Mateo
+Mallett
+Lozada
+Lemoine
+Kuehn
+Hallett
+Grim
+Gillard
+Gaylor
+Garman
+Gallaher
+Feaster
+Faris
+Darrow
+Dardar
+Coney
+Carreon
+Byron
+Braithwaite
+Boylan
+Boyett
+Born
+Bixler
+Bigham
+Benford
+Barragan
+Barnum
+Zuber
+Wyche
+Westcott
+Vining
+Stoltzfus
+Simonds
+Shupe
+Sabin
+Ruble
+Rittenhouse
+Richman
+Perrone
+Mulholland
+Millan
+Meister
+Mathew
+Lomeli
+Kite
+Jemison
+Hulett
+Holler
+Hickerson
+Herold
+Hazelwood
+Griffen
+Gause
+Forde
+Eisenberg
+Dilworth
+Charron
+Chaisson
+Brodie
+Bristow
+Breunig
+Brace
+Boutwell
+Bentz
+Belk
+Bayless
+Batchelder
+Baran
+Baeza
+Zimmermann
+Weathersby
+Volk
+Toole
+Theis
+Tedesco
+Shine
+Searle
+Schenck
+Satterwhite
+Sandy
+Ruelas
+Royce
+Rankins
+Partida
+Nesbit
+Morel
+Menchaca
+Levasseur
+Kaylor
+Johnstone
+Hulse
+Hollar
+Hersey
+Harrigan
+Harbison
+Guyer
+Gish
+Giese
+Gerlach
+Geller
+Geisler
+Falcone
+Ernest
+Elwell
+Doucet
+Deese
+Darr
+Corder
+Chafin
+Byler
+Bussell
+Burdett
+Brasher
+Bowe
+Bellinger
+Bastian
+Barner
+Alleyne
+Wilborn
+Weil
+Wegner
+Wales
+Tatro
+Spitzer
+Smithers
+Schoen
+Resendez
+Pete
+Parisi
+Overman
+Obrian
+Mudd
+Moy
+Mclaren
+Mahler
+Maggio
+Lindner
+Lalonde
+Lacasse
+Laboy
+Killion
+Kahl
+Jessen
+Jamerson
+Houk
+Henshaw
+Gustin
+Groom
+Graber
+Durst
+Duenas
+Davey
+Cundiff
+Conlon
+Colunga
+Coakley
+Chiles
+Capers
+Buell
+Bricker
+Bissonnette
+Birmingham
+Bartz
+Bagby
+Zayas
+Volpe
+Treece
+Toombs
+Thom
+Terrazas
+Swinney
+Skiles
+Silveira
+Shouse
+Senn
+Rambo
+Ramage
+Nez
+Moua
+Marlin
+Malik
+Langham
+Kyles
+Holston
+Hoagland
+Herd
+Hector
+Feller
+Emory
+Denison
+Corliss
+Carraway
+Burford
+Bickel
+Ambriz
+Abercrombie
+Yamada
+Winner
+Weidner
+Waddle
+Verduzco
+Thurmond
+Swindle
+Schrock
+Sanabria
+Rosenberger
+Probst
+Peabody
+Olinger
+Neighbors
+Nazario
+Mccafferty
+Mcbroom
+Mcabee
+Mazur
+Matherne
+Mapes
+Leverett
+Killingsworth
+Heisler
+Griego
+Grande
+Gosnell
+Frankel
+Franke
+Ferrante
+Fenn
+Elmer
+Ehrlich
+Christopherso
+Chick
+Chasse
+Chancellor
+Caton
+Brunelle
+Bly
+Bloomfield
+Babbitt
+Azevedo
+Abramson
+Ables
+Abeyta
+Youmans
+Wozniak
+Wainwright
+Summer
+Stowell
+Smitherman
+Sites
+Samuelson
+Runge
+Rule
+Rothman
+Rosenfeld
+Quan
+Peake
+Oxford
+Owings
+Olmos
+Munro
+Moreira
+Leatherwood
+Larkins
+Krantz
+Kovacs
+Kizer
+Kindred
+Karnes
+Jaffe
+Hubbell
+Hosey
+Hauck
+Harold
+Goodell
+Favors
+Erdman
+Dvorak
+Doane
+Cureton
+Cofer
+Buehler
+Bierman
+Berndt
+Banta
+Annis
+Abram
+Abdullah
+Warwick
+Waltz
+Turcotte
+Trinh
+Torrey
+Stith
+Seger
+Sachs
+Quesada
+Pinder
+Peppers
+Pascual
+Paschall
+Parkhurst
+Ozuna
+Oster
+Nicholls
+Mortimer
+Lheureux
+Lavalley
+Kimura
+Jablonski
+Haun
+Gourley
+Gilligan
+Fix
+Derby
+Croy
+Cotto
+Cargill
+Burwell
+Burgett
+Buckman
+Brett
+Booher
+Adorno
+Wrenn
+Whittemore
+Urias
+Szabo
+Sayles
+Saiz
+Rutland
+Rael
+Plant
+Pharr
+Penney
+Pelkey
+Ogrady
+Nickell
+Musick
+Moats
+Mather
+Massa
+Laurent
+Kirschner
+Kieffer
+Kellar
+Hendershot
+Gott
+Godoy
+Gadson
+Furtado
+Fiedler
+Erskine
+Edison
+Dutcher
+Dever
+Daggett
+Chevalier
+Chao
+Brake
+Ballesteros
+Amerson
+Alejandro
+Wingo
+Waldon
+Trott
+Spikes
+Silvey
+Showers
+Schlegel
+Rue
+Ritz
+Pepin
+Pelayo
+Parsley
+Palermo
+Moorehead
+Mchale
+Lett
+Kocher
+Kilburn
+Iglesias
+Humble
+Hulbert
+Huckaby
+Hix
+Haven
+Hartford
+Hardiman
+Gurney
+Grigg
+Grasso
+Goings
+Fillmore
+Farber
+Depew
+Dandrea
+Dame
+Cowen
+Covarrubias
+Cory
+Burrus
+Bracy
+Ardoin
+Thompkins
+Suzuki
+Standley
+Russel
+Radcliffe
+Pohl
+Persaud
+Percy
+Parenteau
+Pabon
+Newson
+Newhouse
+Napolitano
+Mulcahy
+Maya
+Malave
+Keim
+Hooten
+Hernandes
+Heffernan
+Hearne
+Greenleaf
+Glick
+Fuhrman
+Fetter
+Faria
+Dishman
+Dickenson
+Crites
+Criss
+Clapper
+Chenault
+Castor
+Casto
+Bugg
+Bove
+Bonney
+Blessing
+Ard
+Anderton
+Allgood
+Alderson
+Woodman
+Wisdom
+Warrick
+Toomey
+Tooley
+Tarrant
+Summerville
+Stebbins
+Sokol
+Sink
+Searles
+Schutz
+Schumann
+Scheer
+Remillard
+Raper
+Proulx
+Palmore
+Monroy
+Miguel
+Messier
+Melo
+Melanson
+Mashburn
+Manzano
+Lussier
+Lovely
+Lien
+Jenks
+Huneycutt
+Hartwig
+Grimsley
+Fulk
+Fielding
+Fidler
+Engstrom
+Eldred
+Dantzler
+Crandell
+Ching
+Calder
+Brumley
+Breton
+Brann
+Bramlett
+Boykins
+Bianco
+Bancroft
+Almaraz
+Alcantar
+Whitmer
+Whitener
+Welton
+Vineyard
+Su
+Rahn
+Paquin
+Mizell
+Mix
+Mcmillin
+Mckean
+Marston
+Maciel
+Lundquist
+Louie
+Liggins
+Lampkin
+Kranz
+Koski
+Kirkham
+Jiminez
+Hazzard
+Harrod
+Graziano
+Grammer
+Gendron
+Garrido
+Fordham
+Englert
+Elwood
+Dryden
+Demoss
+Deluna
+Crabb
+Comeau
+Claudio
+Brummett
+Blume
+Benally
+Wessel
+Vanbuskirk
+Thorson
+Stumpf
+Stockwell
+Rocco
+Reams
+Radtke
+Rackley
+Pelton
+Niemi
+Newland
+Nelsen
+Morrissette
+Miramontes
+Mcginley
+Mccluskey
+Marley
+Marchant
+Luevano
+Lampe
+Lail
+Jeffcoat
+Infante
+Hu
+Hinman
+Gaona
+Erb
+Eady
+Desmarais
+Decosta
+Dansby
+Cisco
+Choe
+Breckenridge
+Bostwick
+Borg
+Bianchi
+Beer
+Alberts
+Adrian
+Wilkie
+Whorton
+Vargo
+Tait
+Sylvia
+Soucy
+Schuman
+Ousley
+Mumford
+Lum
+Lippert
+Leath
+Lavergne
+Laliberte
+Kirksey
+Kenner
+Johnsen
+Izzo
+Hiles
+Gullett
+Greenwell
+Gaspar
+Galbreath
+Gaitan
+Ericson
+Duck
+Delapaz
+Croom
+Cottingham
+Clift
+Bushnell
+Boozer
+Bice
+Bernardo
+Beason
+Arrowood
+Waring
+Voorhees
+Truax
+Shreve
+Shockey
+Schatz
+Sandifer
+Rubino
+Rozier
+Roseberry
+Roll
+Player
+Pieper
+Peden
+Nester
+Nave
+Murphey
+Malinowski
+Macgregor
+Liang
+Lafrance
+Kunkle
+Kirkman
+Jorge
+Hipp
+Hasty
+Haddix
+Gervais
+Gerdes
+Garfield
+Gamache
+Fouts
+Fitzwater
+Dillingham
+Deming
+Deanda
+Cedeno
+Cannady
+Burson
+Bouldin
+Arceneaux
+Woodhouse
+Whitford
+Wescott
+Welty
+Weigel
+Torgerson
+Toms
+Surber
+Sunderland
+Sterner
+Setzer
+Salvatore
+Riojas
+Pumphrey
+Puga
+Pedro
+Patch
+Metts
+Mcgarry
+Mccandless
+Magill
+Lupo
+Loveland
+Llamas
+Leclerc
+Koons
+Kahler
+Huss
+Holbert
+Heintz
+Haupt
+Grimmett
+Gaskill
+Flower
+Ellingson
+Dorr
+Dingess
+Deweese
+Desilva
+Crossley
+Cordeiro
+Converse
+Conde
+Cheeks
+Caldera
+Cairns
+Burmeister
+Burkhalter
+Brawner
+Bott
+Youngs
+Vierra
+Valladares
+Tiffany
+Shrum
+Shropshire
+Sevilla
+Rusk
+Roof
+Rodarte
+Pedraza
+Nino
+Montana
+Merino
+Mcminn
+Markle
+Mapp
+Lucia
+Lajoie
+Koerner
+Kittrell
+Kato
+Hyder
+Hollifield
+Heiser
+Hazlett
+Greenwald
+Fant
+Eldredge
+Dreher
+Delafuente
+Cravens
+Claypool
+Beecher
+Aronson
+Alanis
+Worthen
+Wojcik
+Winger
+Whitacre
+Wellington
+Valverde
+Valdivia
+Troupe
+Thrower
+Swindell
+Suttles
+Suh
+Stroman
+Spires
+Slate
+Shealy
+Sarver
+Sartin
+Sadowski
+Rondeau
+Rolon
+Rick
+Rex
+Rascon
+Priddy
+Pine
+Paulino
+Nolte
+Munroe
+Molloy
+Mellon
+Mciver
+Lykins
+Loggins
+Lillie
+Lenoir
+Klotz
+Kempf
+Jone
+Hupp
+Hollowell
+Hollander
+Haynie
+Hassan
+Harkness
+Harker
+Gottlieb
+Frith
+Eddins
+Driskell
+Doggett
+Densmore
+Charette
+Cassady
+Carrol
+Byrum
+Burcham
+Buggs
+Benn
+Whitted
+Warrington
+Vandusen
+Vaillancourt
+Steger
+Spell
+Siebert
+Scofield
+Quirk
+Purser
+Plumb
+Orcutt
+Northern
+Nordstrom
+Mosely
+Michalski
+Mcphail
+Mcdavid
+Mccraw
+Martini
+Marchese
+Mannino
+Leo
+Lefevre
+Largent
+Lanza
+Kress
+Isham
+Hunsaker
+Hoch
+Hildebrandt
+Guarino
+Grijalva
+Graybill
+Fick
+Ewell
+Ewald
+Deangelo
+Cusick
+Crumley
+Coston
+Cathcart
+Carruthers
+Bullington
+Brian
+Bowes
+Blain
+Blackford
+Barboza
+Yingling
+Woodland
+Wert
+Weiland
+Varga
+Silverstein
+Sievers
+Shuster
+Shumway
+Scudder
+Runnels
+Rumsey
+Renfroe
+Provencher
+Polley
+Mohler
+Middlebrooks
+Kutz
+Koster
+Korn
+Grow
+Groth
+Glidden
+Fazio
+Deen
+Corn
+Copper
+Chipman
+Chenoweth
+Champlin
+Cedillo
+Carrero
+Carmody
+Buckles
+Brien
+Boutin
+Bosch
+Bill
+Berkowitz
+Altamirano
+Wilfong
+Wiegand
+Waites
+Truesdale
+Toussaint
+Tobey
+Tedder
+Steelman
+Sirois
+Schnell
+Robichaud
+Ridge
+Richburg
+Pray
+Plumley
+Pizarro
+Piercy
+Ortego
+Oberg
+Neace
+Music
+Mickey
+Mertz
+Mcnew
+Matta
+Lawyer
+Lapp
+Lair
+Kibler
+Jessie
+Howlett
+Hollister
+Hofer
+Hatten
+Hagler
+Germany
+Falgoust
+Engelhardt
+Eberle
+Eastwood
+Dombrowski
+Dinsmore
+Daye
+Cool
+Casares
+Capone
+Braud
+Balch
+Autrey
+Wendel
+Tyndall
+Toy
+Strobel
+Stoltz
+Spinelli
+Serrato
+Rochester
+Reber
+Real
+Rathbone
+Palomino
+Noah
+Nickels
+Mayle
+Mathers
+Mach
+Loeffler
+Littrell
+Levinson
+Leong
+Lemire
+Lejeune
+Lazo
+Lasley
+Koller
+Kennard
+Jester
+Hoelscher
+Hintz
+Hagerman
+Greaves
+Fore
+Eudy
+Engler
+Corrales
+Cordes
+Brunet
+Bidwell
+Bennet
+Bare
+Tyrrell
+Tharpe
+Swinton
+Stribling
+Steven
+Southworth
+Sisneros
+Shane
+Savoie
+Samons
+Ruvalcaba
+Roscoe
+Ries
+Ramer
+Omara
+Mosqueda
+Millar
+Mcpeak
+Macomber
+Luckey
+Litton
+Lehr
+Lavin
+Hubbs
+Hoard
+Hibbs
+Hagans
+Futrell
+Exum
+Evenson
+Dicks
+Culler
+Chou
+Carbaugh
+Callen
+Brashear
+Bloomer
+Blakeney
+Bigler
+Addington
+Woodford
+Witter
+Unruh
+Tolentino
+Sumrall
+Stgermain
+Smock
+Sherer
+Salem
+Rochelle
+Rayner
+Pooler
+Oquinn
+Nero
+Milano
+Mcglothlin
+Mars
+Linden
+Kowal
+Kerrigan
+Ibrahim
+Harvell
+Hanrahan
+Goodall
+Geist
+Fussell
+Fung
+Ferebee
+Federico
+Eley
+Eggert
+Dorsett
+Dingman
+Destefano
+Colucci
+Clemmer
+Caesar
+Burnell
+Brumbaugh
+Boddie
+Berryhill
+Avelar
+Alcantara
+Abbey
+Winder
+Winchell
+Vandenberg
+Trotman
+Thurber
+Thibeault
+Stlouis
+Stilwell
+Sperling
+Shattuck
+Sarmiento
+Ruppert
+Rumph
+Renaud
+Randazzo
+Rademacher
+Quiles
+Pearman
+Palomo
+Mercurio
+Lowrey
+Lindeman
+Lawlor
+Larosa
+Lander
+Labrecque
+Kimber
+Hovis
+Holifield
+Henninger
+Hawkes
+Hartfield
+Hann
+Hague
+Genovese
+Garrick
+Fudge
+Frink
+Eddings
+Dinh
+Dear
+Cutter
+Cribbs
+Constant
+Calvillo
+Bunton
+Brodeur
+Bolding
+Blanding
+Agosto
+Zahn
+Wiener
+Trussell
+Tew
+Tello
+Teixeira
+Stephan
+Speck
+Sharma
+Shanklin
+Sealy
+Scanlan
+Santamaria
+Roundy
+Robichaux
+Ringer
+Rigney
+Prevost
+Polson
+Philip
+Pass
+Nord
+Moxley
+Mohammed
+Medford
+Mccaslin
+Mcardle
+Macarthur
+Lewin
+Lasher
+Ketcham
+Keiser
+Heine
+Hackworth
+Grose
+Grizzle
+Grass
+Gillman
+Gartner
+Garth
+Frazee
+Fleury
+Fast
+Edson
+Edmonson
+Derry
+Deck
+Cronk
+Conant
+Burress
+Burgin
+Broom
+Brockington
+Bolick
+Boger
+Birchfield
+Billington
+Baily
+Bahena
+Armbruster
+Anson
+Yoho
+Wilcher
+Tinney
+Timberlake
+Thoma
+Thielen
+Sutphin
+Stultz
+Sikora
+Serra
+Schulman
+Scheffler
+Santillan
+Robin
+Rego
+Preciado
+Pinkham
+Monday
+Mickle
+Luu
+Lomas
+Lizotte
+Lent
+Lenard
+Kellerman
+Keil
+Juan
+Johanson
+Hernadez
+Hartsfield
+Hang
+Haber
+Gorski
+Farkas
+Eberhardt
+Duquette
+Delano
+Cropper
+Cozart
+Cockerham
+Chamblee
+Cartagena
+Cahoon
+Buzzell
+Brister
+Brewton
+Blackshear
+Benfield
+Aston
+Ashburn
+Arruda
+Wetmore
+Weise
+Vaccaro
+Tucci
+Sudduth
+Stromberg
+Stoops
+Showalter
+Shears
+Runion
+Rowden
+Rosenblum
+Riffle
+Renfrow
+Peres
+Obryant
+Nicolas
+Leftwich
+Lark
+Landeros
+Kistler
+Killough
+Kerley
+Kastner
+Hoggard
+Hartung
+Guertin
+Govan
+Gatling
+Gailey
+Fullmer
+Fulford
+Flatt
+Esquibel
+Endicott
+Edmiston
+Edelstein
+Dufresne
+Dressler
+Dickman
+Chee
+Busse
+Bonnett
+Bogart
+Berard
+Barrington
+Arena
+Anton
+Yoshida
+Velarde
+Veach
+Vanhouten
+Vachon
+Tolson
+Tolman
+Tennyson
+Stites
+Soler
+Shutt
+Ruggles
+Rhone
+Pegues
+Ong
+Neese
+Muro
+Moncrief
+Mefford
+Mcphee
+Mcmorris
+Mceachern
+Mcclurg
+Mansour
+Mai
+Mader
+Leija
+Lecompte
+Lafountain
+Labrie
+Jaquez
+Heald
+Hash
+Hartle
+Gainer
+Frisby
+Farina
+Eidson
+Edgerton
+Dyke
+Durrett
+Duhon
+Cuomo
+Cobos
+Cervantez
+Bybee
+Brockway
+Borowski
+Binion
+Beery
+Arguello
+Amaro
+Acton
+Yuen
+Winton
+Wigfall
+Weekley
+Vidrine
+Vannoy
+Tardiff
+Shoop
+Shilling
+Schick
+Sand
+Safford
+Prendergast
+Pilgrim
+Pellerin
+Osuna
+Nissen
+Nalley
+Moritz
+Moller
+Messner
+Messick
+Merry
+Merrifield
+Mcguinness
+Matherly
+Marcano
+Mahone
+Lemos
+Lebrun
+Jara
+Hoffer
+Hewlett
+Herren
+Hecker
+Haws
+Haug
+Hack
+Gwin
+Gober
+Gilliard
+Fredette
+Favela
+Echeverria
+Downer
+Donofrio
+Desrochers
+Dee
+Crozier
+Corson
+Clyde
+Bechtold
+Argueta
+Aparicio
+Zamudio
+Willette
+Westover
+Westerman
+Utter
+Troyer
+Thies
+Tapley
+Slavin
+Shirk
+Sandler
+Roop
+Rimmer
+Raymer
+Range
+Radcliff
+Otten
+Moorer
+Millet
+Mckibben
+Mccutchen
+Mcavoy
+Mcadoo
+Mayorga
+Mastin
+Martineau
+Marek
+Madore
+Leflore
+Kroeger
+Kennon
+Jimerson
+Javier
+Hostetter
+Hornback
+Hendley
+Hance
+Guardado
+Granado
+Gowen
+Goodale
+Flinn
+Fleetwood
+Fitz
+Durkee
+Duprey
+Dipietro
+Dilley
+Clyburn
+Brawley
+Beckley
+Arana
+Weatherby
+Vollmer
+Victoria
+Vestal
+Tunnell
+Trigg
+Tingle
+Takahashi
+Sweatt
+Storer
+Snapp
+Shiver
+Rooker
+Red
+Rathbun
+Poisson
+Perrine
+Perri
+Pastor
+Parmer
+Parke
+Pare
+Papa
+Palmieri
+Nottingham
+Midkiff
+Mecham
+Mccomas
+Mcalpine
+Lovelady
+Lillard
+Lally
+Knopp
+Kile
+Kiger
+Haile
+Gupta
+Goldsberry
+Gilreath
+Fulks
+Friesen
+Franzen
+Flack
+Findlay
+Ferland
+Dreyer
+Dore
+Dennard
+Deckard
+Debose
+Crim
+Coulombe
+Cork
+Chancey
+Cantor
+Branton
+Bissell
+Barns
+Woolard
+Witham
+Wasserman
+Waldo
+Spiegel
+Shoffner
+Scholz
+Ruch
+Rossman
+Ready
+Petry
+Palacio
+Paez
+Neary
+Mortenson
+Millsap
+Miele
+Mick
+Menke
+Mckim
+Mcanally
+Martines
+Manor
+Malcom
+Lemley
+Larochelle
+Klaus
+Klatt
+Kaufmann
+Kapp
+Helmer
+Hedge
+Halloran
+Glisson
+Frechette
+Fontana
+Enoch
+Eagan
+Drum
+Distefano
+Danley
+Creekmore
+Chartier
+Chaffee
+Carillo
+Burg
+Bolinger
+Berkley
+Benz
+Basso
+Bash
+Barrier
+Zelaya
+Woodring
+Witkowski
+Wilmot
+Wilkens
+Wieland
+Virgil
+Verdugo
+Urquhart
+Tsai
+Timms
+Swiger
+Swaim
+Sussman
+Scarlett
+Pires
+Molnar
+Mcatee
+Maurice
+Lowder
+Loos
+Linker
+Landes
+Kingery
+Keeley
+Hufford
+Higa
+Hendren
+Hammack
+Hamann
+Gillam
+Gerhardt
+Fell
+Eugene
+Edelman
+Eby
+Delk
+Deans
+Curl
+Constantine
+Cleaver
+Claar
+Casiano
+Carruth
+Carlyle
+Bump
+Brophy
+Bolanos
+Bibbs
+Bessette
+Beggs
+Baugher
+Bartel
+Averill
+Andresen
+Amin
+Alden
+Adames
+Wildman
+Via
+Valente
+Turnbow
+Tse
+Swink
+Sublett
+Stroh
+Stringfellow
+Ridgway
+Pugliese
+Poteat
+Pang
+Ohare
+Neubauer
+Murchison
+Mohamed
+Mingo
+Lucky
+Lemmons
+Kwon
+Kellam
+Kean
+Jarmon
+Hyden
+Hudak
+Hollinger
+Henkel
+Hemingway
+Hasson
+Hansel
+Halter
+Haire
+Goodnight
+Ginsberg
+Gillispie
+Fogel
+Flory
+Etter
+Elledge
+Eckman
+Deas
+Currin
+Crafton
+Coomer
+Colter
+Claxton
+Bulter
+Braddock
+Bowyer
+Blizzard
+Binns
+Bing
+Bellows
+Baskerville
+Barros
+Ansley
+Woolf
+Wight
+Waldman
+Wadley
+Tull
+Trull
+Tesch
+Struck
+Stouffer
+Stadler
+Slay
+Shubert
+Sedillo
+Santacruz
+Reinke
+Raleigh
+Poynter
+Neri
+Neale
+Natividad
+Mowry
+Moralez
+Monger
+Mitchum
+Merryman
+Manion
+Macdougall
+Lux
+Litchfield
+Ley
+Levitt
+Lepage
+Lasalle
+Laine
+Khoury
+Kavanagh
+Karns
+Ivie
+Huebner
+Hodgkins
+Halpin
+Garica
+Eversole
+Dutra
+Dunagan
+Duffey
+Dillman
+Dillion
+Deville
+Dearborn
+Damato
+Courson
+Coulson
+Burdine
+Bryce
+Bousquet
+Bonin
+Bish
+Atencio
+Westbrooks
+Wages
+Vaca
+Tye
+Toner
+Tomas
+Tillis
+Swett
+Surface
+Struble
+Stanfill
+Son
+Solorzano
+Slusher
+Sipple
+Sim
+Silvas
+Shults
+Schexnayder
+Saez
+Rodas
+Rager
+Pulver
+Plaza
+Penton
+Paniagua
+Meneses
+Mcfarlin
+Mcauley
+Matz
+Maloy
+Magruder
+Lohman
+Landa
+Lacombe
+Jaimes
+Hom
+Holzer
+Holst
+Heil
+Hackler
+Grundy
+Gregor
+Gilkey
+Farnham
+Durfee
+Dunton
+Dunston
+Duda
+Dews
+Dana
+Craver
+Corriveau
+Conwell
+Colella
+Chambless
+Bremer
+Boutte
+Bourassa
+Blaisdell
+Backman
+Babineaux
+Audette
+Alleman
+Towner
+Taveras
+Tarango
+Sullins
+Suiter
+Stallard
+Solberg
+Schlueter
+Poulos
+Pimental
+Owsley
+Olivier
+Okelley
+Nations
+Moffatt
+Metcalfe
+Meekins
+Medellin
+Mcglynn
+Mccowan
+Marriott
+Marable
+Lennox
+Lamoureux
+Koss
+Kerby
+Karp
+Jason
+Isenberg
+Howze
+Hockenberry
+Highsmith
+Harbour
+Hallmark
+Gusman
+Greeley
+Giddings
+Gaudet
+Gallup
+Fleenor
+Eicher
+Edington
+Dimaggio
+Dement
+Demello
+Decastro
+Cruise
+Bushman
+Brundage
+Brooker
+Brooke
+Bourg
+Board
+Blackstock
+Bergmann
+Beaton
+Banister
+Argo
+Appling
+Wortman
+Watterson
+Villalpando
+Tillotson
+Tighe
+Sundberg
+Sternberg
+Stamey
+Speaks
+Shipe
+Seeger
+Scarberry
+Sattler
+Sain
+Rothstein
+Poteet
+Plowman
+Pettiford
+Penland
+Peach
+Partain
+Pankey
+Oyler
+Ogletree
+Ogburn
+Moton
+Million
+Merkel
+Mask
+Markus
+Lucier
+Lazarus
+Lavelle
+Lakey
+Kratz
+Kinser
+Kershaw
+Josephson
+Jesse
+Imhoff
+Ibanez
+Hendry
+Hammon
+Frisbie
+Friedrich
+Frawley
+Fraga
+Forester
+Eskew
+Emmert
+Drennan
+Doyon
+Dominick
+Dandridge
+Cumming
+Cawley
+Carvajal
+Bracey
+Belisle
+Batey
+Ahner
+Wysocki
+Weiser
+Veliz
+Tincher
+Sherlock
+Santo
+Sansone
+Sankey
+Sandstrom
+Sale
+Rohrer
+Risner
+Pridemore
+Pfeffer
+Persinger
+Peery
+Oubre
+Orange
+Nowicki
+Musgrave
+Murdoch
+Mullinax
+Mccary
+Mathieu
+Livengood
+Leonardo
+Kyser
+Klink
+Kimes
+Kellner
+Kavanaugh
+Kasten
+Imes
+Hoey
+Hinshaw
+Halley
+Hake
+Gurule
+Grube
+Grillo
+Geter
+Gatto
+Garver
+Garretson
+Farwell
+Eiland
+Dunford
+Decarlo
+Corso
+Core
+Colman
+Collard
+Cleghorn
+Chasteen
+Cavender
+Carlile
+Calvo
+Byerly
+Brogdon
+Broadwater
+Breault
+Bono
+Bergin
+Behr
+Ballenger
+Amick
+Yan
+Vice
+Tamez
+Stiffler
+Steinke
+Simmon
+Shankle
+Schaller
+Salmons
+Sackett
+Saad
+Rideout
+Reader
+Ratcliffe
+Rao
+Ranson
+Randell
+Plascencia
+Petterson
+Olszewski
+Olney
+Olguin
+Nilsson
+Nevels
+Morelli
+Montiel
+Monge
+Michell
+Michaelson
+Mertens
+Mcchesney
+Mcalpin
+Mathewson
+Lower
+Loudermilk
+Lineberry
+Liggett
+Lamp
+Kinlaw
+Kight
+Just
+Jost
+Hereford
+Hardeman
+Halpern
+Halliday
+Hafer
+Gaul
+Friel
+Freitag
+Frances
+Forsberg
+Evangelista
+Doering
+Dicarlo
+Dendy
+Delp
+Deguzman
+Dameron
+Curtiss
+Cousin
+Cosper
+Charley
+Cauthen
+Cao
+Camper
+Bradberry
+Bouton
+Bonnell
+Bixby
+Bieber
+Beveridge
+Belle
+Bedwell
+Barhorst
+Bannon
+Baltazar
+Baier
+Ayotte
+Attaway
+Arenas
+Alex
+Abrego
+Watford
+Valley
+Turgeon
+Tunstall
+Thaxton
+Thai
+Tenorio
+Stotts
+Sthilaire
+Spiker
+Shedd
+Seng
+Seabolt
+Scalf
+Salyers
+Ruhl
+Rowlett
+Robinett
+Pfister
+Perlman
+Pepe
+Parkman
+Paradise
+Olin
+Nunnally
+Norvell
+Napper
+Modlin
+Mckellar
+Mcclean
+Mascarenas
+Manchester
+Leibowitz
+Ledezma
+Kuhlman
+Kobayashi
+Hunley
+Holmquist
+Hinkley
+Hazard
+Hartsell
+Gribble
+Gravely
+Fifield
+Eliason
+Doctor
+Doak
+Crossland
+Cover
+Clair
+Carleton
+Butters
+Bridgeman
+Bojorquez
+Boggess
+Banker
+Auten
+Woosley
+Wine
+Whiteley
+Wexler
+Twomey
+Tullis
+Townley
+To
+Standridge
+Stamp
+Springs
+Santoyo
+Rueda
+Riendeau
+Revell
+Pless
+Ottinger
+Nigro
+Nickles
+Mulvey
+Menefee
+Mcshane
+Mcloughlin
+Mckinzie
+Marrow
+Markey
+Mariano
+Lockridge
+Lipsey
+Knisley
+Knepper
+Kitts
+Kiel
+Jinks
+Hathcock
+Godin
+Gallego
+Fikes
+Fecteau
+Estabrook
+Ellinger
+Dustin
+Dunlop
+Dudek
+Diego
+Countryman
+Chauvin
+Chatham
+Bullins
+Brownfield
+Boughton
+Bloodworth
+Bibb
+Baucom
+Barbieri
+Aubin
+Armitage
+Alessi
+Absher
+Abbate
+Zito
+Woolery
+Wiggs
+Wacker
+Violette
+Tynes
+Tolle
+Telles
+Tarter
+Swarey
+Strode
+Stockdale
+Stella
+Stalnaker
+Spina
+Schiff
+Saari
+Risley
+Reading
+Rameriz
+Rakes
+Pettaway
+Penner
+Paulus
+Palladino
+Omeara
+Montelongo
+Melnick
+Mehta
+Mcgary
+Mccourt
+Mccollough
+Marchetti
+Manzanares
+Lowther
+Leiva
+Lauderdale
+Lafontaine
+Kowalczyk
+Knighton
+Joubert
+Jaworski
+Ide
+Huth
+Hurdle
+Hung
+Housley
+Hackman
+Gulick
+Gordy
+Gilstrap
+Gehrke
+Gebhart
+Gaudette
+Foxworth
+Finger
+Essex
+Endres
+Dunkle
+Clare
+Cimino
+Cardinal
+Caddell
+Brauer
+Braley
+Bodine
+Blackmore
+Belden
+Backer
+Ayer
+Andress
+Alva
+Wisner
+Walk
+Vuong
+Valliere
+Twigg
+Tso
+Tavarez
+Strahan
+Steib
+Staub
+Sowder
+Shoulders
+Seiber
+Schutt
+Scharf
+Schade
+Rodriques
+Risinger
+Renshaw
+Rath
+Rahman
+Presnell
+Pillow
+Piatt
+Pasquale
+Nieman
+Nicol
+Nevins
+Milford
+Mcilwain
+Mcgaha
+Mccully
+Mccomb
+Maye
+Massengale
+Macedo
+Lines
+Lesher
+Leland
+Kearse
+Jauregui
+Husted
+Hudnall
+Holmberg
+Hertel
+Hershey
+Hardie
+Glidewell
+Frausto
+Fassett
+Dash
+Dalessandro
+Dahlgren
+Corum
+Constantino
+Conlin
+Colquitt
+Colombo
+Claycomb
+Carley
+Cardin
+Cancel
+Buller
+Boring
+Boney
+Bocanegra
+Blazer
+Biggers
+Benedetto
+Araiza
+Andino
+Albin
+Zorn
+Werth
+Weisman
+Walley
+Vanegas
+Ulibarri
+Towers
+Towe
+Tedford
+Teasley
+Suttle
+Steffens
+Stcyr
+Squire
+Smythe
+Singley
+Sifuentes
+Shuck
+Session
+Schram
+Sass
+Rieger
+Ridenhour
+Rickert
+Richerson
+Rayborn
+Rabe
+Raab
+Pendley
+Pastore
+Ordway
+Moynihan
+Mellott
+Mckissick
+Mcgann
+Mccready
+Mauney
+Marrufo
+List
+Lenhart
+Lazar
+Lafave
+Keele
+Kautz
+Jardine
+Jahnke
+Jacobo
+Hord
+Hardcastle
+Hageman
+Griffey
+Giglio
+Gehring
+Fortson
+Duque
+Duplessis
+Donner
+Dicken
+Derosier
+Deitz
+Dalessio
+Cyrus
+Cram
+Chi
+Center
+Castleman
+Candelario
+Callison
+Caceres
+Bozarth
+Biles
+Bejarano
+Beech
+Bashaw
+Avina
+Armentrout
+Angus
+Alverez
+Acord
+Zack
+Waterhouse
+Vereen
+Vanlandingham
+Uhl
+Strawser
+Shotwell
+Severance
+Seltzer
+Schoonmaker
+Schock
+Schaub
+Schaffner
+Roeder
+Rodrigez
+Riffe
+Rhine
+Rasberry
+Rancourt
+Railey
+Quade
+Pursley
+Prouty
+Perdomo
+Oxley
+Osterman
+Nickens
+Murphree
+Mounts
+Monte
+Merida
+Maus
+Mattern
+Masse
+Martinelli
+Mangan
+Lutes
+Ludwick
+Loney
+Laureano
+Lasater
+Knighten
+Kissinger
+Kimsey
+Kessinger
+Honea
+Hollingshead
+Hockett
+Heyer
+Heron
+Gurrola
+Gove
+Glasscock
+Gillett
+Galan
+Featherstone
+Eckhardt
+Duron
+Dunson
+Dasher
+Culbreth
+Cowden
+Cowans
+Claypoole
+Churchwell
+Chabot
+Caviness
+Cater
+Caston
+Callan
+Byington
+Burkey
+Boden
+Beckford
+Atwater
+Arms
+Archambault
+Alvey
+Alsup
+Yon
+Whisenant
+Weese
+Voyles
+Verret
+Tsang
+Tessier
+Sweitzer
+Sherwin
+Shaughnessy
+Revis
+Remy
+Prine
+Philpott
+Peavy
+Paynter
+Parmenter
+Ovalle
+Offutt
+Nightingale
+Newlin
+Nakano
+Myatt
+Muth
+Mohan
+Mcmillon
+Mccarley
+Mccaleb
+Maxson
+Marinelli
+Maley
+Macy
+Liston
+Letendre
+Kain
+Huntsman
+Hirst
+Hagerty
+Gulledge
+Greenway
+Grajeda
+Gorton
+Goines
+Gittens
+Frederickson
+Fanelli
+Embree
+Eichelberger
+Dunkin
+Dull
+Dixson
+Dillow
+Defelice
+Chumley
+Burleigh
+Borkowski
+Binette
+Biggerstaff
+Berglund
+Beller
+Audet
+Arbuckle
+Allain
+Alfano
+Zander
+Youngman
+Wittman
+Weintraub
+Vanzant
+Vaden
+Twitty
+Trader
+Toon
+Till
+Stollings
+Standifer
+Spinner
+Sines
+Shope
+Scalise
+Saville
+Romans
+Posada
+Pisano
+Otte
+Nolasco
+Napoli
+Mier
+Merkle
+Mendiola
+Melcher
+Mejias
+Mcmurry
+Mccalla
+Markowitz
+Marine
+Manis
+Mallette
+Macfarlane
+Lough
+Looper
+Landin
+Kittle
+Kinsella
+Kinnard
+Hobart
+Herald
+Helman
+Hellman
+Hartsock
+Halford
+Hage
+Gordan
+Glasser
+Gayton
+Gattis
+Gastelum
+Gaspard
+Frisch
+Force
+Fitzhugh
+Eckstein
+Eberly
+Dowden
+Despain
+Crumpler
+Crotty
+Cornelison
+Collin
+Colin
+Chouinard
+Chamness
+Catlin
+Cann
+Bumgardner
+Budde
+Branum
+Bradfield
+Braddy
+Borst
+Birdwell
+Bent
+Bazan
+Bank
+Banas
+Bade
+Aubrey
+Arango
+Ahearn
+Addis
+Zumwalt
+Wurth
+Wilk
+Widener
+Wagstaff
+Vella
+Urrutia
+Terwilliger
+Tart
+Steinman
+Staats
+Sloat
+Rives
+Riggle
+Revels
+Reichard
+Prickett
+Poff
+Pitzer
+Petro
+Pell
+Northrup
+Nicks
+Moline
+Mielke
+Maynor
+Mallon
+Magness
+Lingle
+Lindell
+Lieb
+Lesko
+Lebeau
+Lammers
+Lafond
+Kiernan
+Ketron
+Jurado
+Holmgren
+Hilburn
+Hayashi
+Hashimoto
+Harbaugh
+Hans
+Guillot
+Gard
+Froehlich
+Felipe
+Feinberg
+Falco
+Dufour
+Drees
+Doney
+Diep
+Delao
+Daves
+Dail
+Cutting
+Crowson
+Coss
+Congdon
+Carner
+Camarena
+Butterworth
+Burlingame
+Bouffard
+Bloch
+Bilyeu
+Barta
+Bakke
+Baillargeon
+Avent
+Aquilar
+Ake
+Aho
+Zeringue
+Yeh
+Yarber
+Wolfson
+Wendell
+Vogler
+Voelker
+Truss
+Troxell
+Thrift
+Strouse
+Spielman
+Sistrunk
+Shows
+Sevigny
+Schuller
+Schaaf
+Ruffner
+Routh
+Roseman
+Ricciardi
+Peraza
+Pegram
+Overturf
+Olander
+Odaniel
+Neu
+Millner
+Melchor
+Maxie
+Marvel
+Maroney
+Machuca
+Macaluso
+Livesay
+Layfield
+Laskowski
+Kwiatkowski
+Ko
+Kiley
+Kilby
+Julien
+Hovey
+Heywood
+Hayman
+Havard
+Harville
+Haigh
+Hagood
+Grieco
+Glassman
+Gebhardt
+Garry
+Freeze
+Fleischer
+Fann
+Elson
+Eccles
+Cunha
+Crumb
+Crew
+Blakley
+Bardwell
+Abshire
+Woodham
+Wines
+Welter
+Wargo
+Varnado
+Tutt
+Traynor
+Swaney
+Svoboda
+Stricker
+Stoffel
+Stambaugh
+Sickler
+Shackleford
+Selman
+Seaver
+Sansom
+Sanmiguel
+Royston
+Rourke
+Rockett
+Rioux
+Puleo
+Pitchford
+Persons
+Normand
+Nardi
+Mulvaney
+Middaugh
+Manners
+Malek
+Lodge
+Leos
+Lathan
+Kujawa
+Kimbro
+Killebrew
+Joshua
+Houlihan
+Hobby
+Hinckley
+Herod
+Hepler
+Hamner
+Hammel
+Hallowell
+Gonsalez
+Gingerich
+Gambill
+Funkhouser
+Fricke
+Fewell
+Falkner
+Endsley
+Dulin
+Drennen
+Deaver
+Dambrosio
+Clover
+Chadwell
+Ceasar
+Castanon
+Canon
+Burkes
+Brune
+Brisco
+Brinker
+Bowker
+Boldt
+Berner
+Bee
+Beaumont
+Beaird
+Bazemore
+Barrick
+Arnette
+Albano
+Younts
+Wunderlich
+Weidman
+Vanness
+Tu
+Toland
+Theobald
+Stickler
+Steiger
+Stanger
+Spies
+Spector
+Sollars
+Smedley
+Seibel
+Scoville
+Saito
+Rye
+Rummel
+Rude
+Rowles
+Rouleau
+Roos
+Rogan
+Roemer
+Ream
+Raya
+Purkey
+Priester
+Perreira
+Penick
+Paulin
+Parkins
+Overcash
+Oleson
+Nicely
+Neves
+Muldrow
+Minard
+Midgett
+Michalak
+Melgar
+Mcentire
+Mcauliffe
+Marti
+Marte
+Lydon
+Lindholm
+Leyba
+Leader
+Langevin
+Lagasse
+Lafayette
+Kesler
+Kelton
+Kao
+Kaminsky
+Jump
+Jaggers
+Humbert
+Huck
+Howarth
+Hinrichs
+Higley
+Gupton
+Guimond
+Gravois
+Giguere
+Fretwell
+Fontes
+Feeley
+Faucher
+Fall
+Evan
+Eichhorn
+Ecker
+Earp
+Dole
+Dinger
+Derryberry
+Demars
+Deel
+Copenhaver
+Collinsworth
+Colangelo
+Cloyd
+Claiborne
+Caulfield
+Carlsen
+Calzada
+Caffey
+Broadus
+Brenneman
+Bouie
+Bodnar
+Blaney
+Blanc
+Blades
+Beltz
+Behling
+Begin
+Barahona
+Yun
+Yockey
+Winkle
+Windom
+Wimer
+Wilford
+Wash
+Villatoro
+Trexler
+Teran
+Taliaferro
+Sydnor
+Swinson
+Snelling
+Smtih
+Siu
+Simonton
+Simoneaux
+Simoneau
+Sherrer
+Seavey
+Scheel
+Rushton
+Rupe
+Ruano
+Rodney
+Rippy
+Reiner
+Reiff
+Rabinowitz
+Quach
+Penley
+Odle
+Nock
+Minnich
+Mckown
+Mccarver
+Mcandrew
+Longley
+Laux
+Lamothe
+Lafreniere
+Kropp
+Krick
+Kates
+Jepson
+Huie
+Howse
+Howie
+Henriques
+Haydon
+Haught
+Hatter
+Hartzog
+Harkey
+Grimaldo
+Goshorn
+Gormley
+Gluck
+Gilroy
+Gillenwater
+Giffin
+Folks
+Fluker
+Feder
+Eyre
+Eshelman
+Eakins
+Dryer
+Disney
+Detwiler
+Delrosario
+Davisson
+Celestine
+Catalan
+Canning
+Calton
+Buster
+Brammer
+Botelho
+Blakney
+Bartell
+Averett
+Askins
+Aker
+Zak
+Worcester
+Witmer
+Wiser
+Winkelman
+Widmer
+Whittier
+Western
+Weitzel
+Wardell
+Wagers
+Ullman
+Tupper
+Tingley
+Tilghman
+Talton
+Simard
+Seda
+Scheller
+Sala
+Rundell
+Rost
+Roa
+Ribeiro
+Rabideau
+Primm
+Porch
+Polite
+Pinon
+Peart
+Ostrom
+Ober
+Nystrom
+Nussbaum
+Nurse
+Naughton
+Murr
+Moorhead
+Monti
+Monteiro
+Melson
+Meissner
+Mclin
+Mcgruder
+Marotta
+Makowski
+Majewski
+Madewell
+Lunt
+Lukens
+Leininger
+Lebel
+Lakin
+Laguna
+Kepler
+Jaques
+Hunnicutt
+Hungerford
+Hoopes
+Hertz
+Heins
+Hammers
+Halliburton
+Grosso
+Gravitt
+Glasper
+Gideon
+Gallman
+Gallaway
+Funke
+Fulbright
+Falgout
+Eakin
+Dostie
+Dorado
+Dewberry
+Derose
+Cutshall
+Crampton
+Costanzo
+Colletti
+Cloninger
+Claytor
+Chiang
+Canterbury
+Campagna
+Burd
+Brokaw
+Broaddus
+Bretz
+Brainard
+Binford
+Bilbrey
+Alpert
+Aitken
+Ahlers
+Zajac
+Yale
+Woolfolk
+Witten
+Windle
+Wayland
+Tramel
+Tittle
+Talavera
+Suter
+Straley
+Stetson
+Specht
+Sommerville
+Soloman
+So
+Skeens
+Sigman
+Sibert
+Shavers
+Schuck
+Schmit
+Sartain
+Sabol
+Rosenblatt
+Rollo
+Rashid
+Rabb
+Province
+Polston
+Nyberg
+Northrop
+Navarra
+Muldoon
+Mulder
+Mikesell
+Mcdougald
+Mcburney
+Mauricio
+Mariscal
+Lui
+Lozier
+Lingerfelt
+Legere
+Latour
+Lagunas
+Lacour
+Kurth
+Ku
+Killen
+Kiely
+Kayser
+Kahle
+Julius
+Isley
+Huertas
+Hower
+Hinz
+Haugh
+Gumm
+Given
+Galicia
+Fortunato
+Flake
+Dunleavy
+Duggins
+Doby
+Digiovanni
+Devaney
+Deltoro
+Cribb
+Crank
+Corpuz
+Coronel
+Comfort
+Coen
+Charbonneau
+Caine
+Burchette
+Blakey
+Blakemore
+Bergquist
+Beene
+Beaudette
+Bayles
+Ballance
+Bakker
+Bailes
+Asberry
+Arwood
+Zucker
+Willman
+Whitesell
+Wald
+Walcott
+Vancleave
+Trump
+Trail
+Strasser
+Simas
+Shorts
+Shick
+Schleicher
+Schaal
+Saleh
+Rotz
+Resnick
+Raphael
+Rainer
+Partee
+Ollis
+Oller
+Oday
+Noles
+Munday
+Mountain
+Mong
+Millican
+Merwin
+Mazzola
+Mansell
+Magallanes
+Llanes
+Lewellen
+Lepore
+Kisner
+Keesee
+Jim
+Jeanlouis
+Ingham
+Hornbeck
+Hermes
+Hawn
+Hartz
+Harber
+Haffner
+Gutshall
+Guth
+Grays
+Grams
+Gowan
+Finlay
+Finkelstein
+Eyler
+Enloe
+Dungan
+Diez
+Dearman
+Dann
+Cull
+Crosson
+Creek
+Chronister
+Cassity
+Campion
+Callihan
+Butz
+Breazeale
+Blumenthal
+Billy
+Berkey
+Batty
+Batton
+Barge
+Arvizu
+Alexis
+Alderete
+Aldana
+Albaugh
+Abernethy
+Work
+Wolter
+Wille
+Tweed
+Tollefson
+Thomasson
+Teter
+Testerman
+Sproul
+Spates
+Southwick
+Soukup
+Skelly
+Senter
+Sealey
+Sawicki
+Sargeant
+Rossiter
+Rosemond
+Repp
+Pound
+Pink
+Pifer
+Ormsby
+Nickelson
+Naumann
+Morabito
+Monzon
+Millsaps
+Millen
+Mcelrath
+Marcoux
+Mantooth
+Madson
+Macneil
+Mackinnon
+Louque
+Leister
+Lampley
+Kushner
+Krouse
+Kirwan
+June
+Jessee
+Janson
+Jahn
+Jacquez
+Islas
+Hutt
+Holladay
+Hillyer
+Hepburn
+Hensel
+Harrold
+Guadalupe
+Gingrich
+Geis
+Gales
+Fults
+Finnell
+Ferri
+Featherston
+Epley
+Ebersole
+Eames
+Dunigan
+Drye
+Dismuke
+Devaughn
+Delorenzo
+Damiano
+Confer
+Collum
+Clower
+Clow
+Claussen
+Clack
+Caylor
+Cawthon
+Casias
+Carreno
+Carlo
+Bluhm
+Bingaman
+Bewley
+Belew
+Beckner
+Beamer
+Barefoot
+Auld
+Amey
+Wolfenbarger
+Wilkey
+Wicklund
+Waltman
+Villalba
+Valero
+Valdovinos
+Ung
+Ullrich
+Tyus
+Twyman
+Trost
+Tardif
+Tanguay
+Stripling
+Steinbach
+Shumpert
+Sasaki
+Sappington
+Sandusky
+Reinhold
+Reinert
+Quijano
+Pye
+Poor
+Placencia
+Pinkard
+Phinney
+Perrotta
+Pernell
+Parrett
+Oxendine
+Owensby
+Orman
+Nuno
+Mori
+Mcroberts
+Mcneese
+Mckamey
+Mccullum
+Markel
+Mardis
+Maines
+Lueck
+Lubin
+Lefler
+Leffler
+Lavery
+Larios
+Labarbera
+Kershner
+Josey
+Jeanbaptiste
+Izaguirre
+Hermosillo
+Haviland
+Hartshorn
+Hamlet
+Hafner
+Ginter
+Getty
+Franck
+Fiske
+Emmett
+Dufrene
+Doody
+Davie
+Dangerfield
+Dahlberg
+Cuthbertson
+Crone
+Coffelt
+Claus
+Chidester
+Chesson
+Cauley
+Caudell
+Cantara
+Campo
+Caines
+Bullis
+Bucci
+Brochu
+Bosco
+Bogard
+Bickerstaff
+Benning
+Arzola
+Antonelli
+Adkinson
+Zellers
+Wulf
+Worsley
+Woolridge
+Whitton
+Westerfield
+Walczak
+Vassar
+Truett
+Trueblood
+Trawick
+Townsley
+Topping
+Tobar
+Telford
+Sung
+Steverson
+Stagg
+Sitton
+Sill
+Sherrell
+Sergent
+Schoenfeld
+Sarabia
+Rutkowski
+Rubenstein
+Rigdon
+Prentiss
+Pomerleau
+Plumlee
+Phoenix
+Philbrick
+Peer
+Patty
+Patnode
+Oloughlin
+Obregon
+Nuss
+Napoleon
+Morell
+Moose
+Mikell
+Mele
+Mcinerney
+Mcguigan
+Mcbrayer
+Lore
+Lor
+Look
+Lollar
+Lakes
+Kuehl
+Kinzer
+Kamp
+Joplin
+Jacobi
+Howells
+Holstein
+Hedden
+Hassler
+Harty
+Halle
+Greig
+Granville
+Gouge
+Goodrum
+Gerhart
+Geier
+Geddes
+Gast
+Forehand
+Ferree
+Fendley
+Feltner
+Fang
+Esqueda
+Encarnacion
+Eichler
+Egger
+Edmundson
+Eatmon
+Dragon
+Doud
+Donohoe
+Donelson
+Dilorenzo
+Digiacomo
+Diggins
+Delozier
+Dejong
+Danford
+Crippen
+Coppage
+Cogswell
+Clardy
+Cioffi
+Cabe
+Brunette
+Bresnahan
+Bramble
+Blomquist
+Blackstone
+Biller
+Bevis
+Bevan
+Bethune
+Benbow
+Baty
+Basinger
+Balcom
+Andes
+Aman
+Aguero
+Adkisson
+Yandell
+Wilds
+Whisenhunt
+Weigand
+Weeden
+Voight
+Villar
+Trottier
+Tillett
+Suazo
+Setser
+Scurry
+Schuh
+Schreck
+Schauer
+Samora
+Roane
+Rinker
+Reimers
+Reason
+Ratchford
+Popovich
+Parkin
+Nichol
+Natal
+Melville
+Mcbryde
+Magdaleno
+Loehr
+Lockman
+Lingo
+Leduc
+Larocca
+Lao
+Lamere
+Laclair
+Krall
+Korte
+Koger
+Jumper
+Jalbert
+Hughs
+Higbee
+Henton
+Heaney
+Haith
+Gump
+Greeson
+Goodloe
+Gholston
+Gasper
+Gagliardi
+Fregoso
+Farthing
+Fabrizio
+Ensor
+Elswick
+Elgin
+Eklund
+Eaddy
+Drouin
+Dorton
+Dizon
+Derouen
+Delia
+Deherrera
+Davy
+Dark
+Dampier
+Cullum
+Culley
+Cowgill
+Cardoso
+Cardinale
+Brodsky
+Broadbent
+Brimmer
+Briceno
+Branscum
+Bolyard
+Boley
+Bennington
+Beadle
+Baur
+Ballentine
+Azure
+Aultman
+Augustus
+Asuncion
+Arciniega
+Aguila
+Aceves
+Yepez
+Yap
+Woodrum
+Wethington
+Weissman
+Veloz
+Trusty
+Troup
+Trammel
+Theodore
+Tarpley
+Stivers
+Steck
+Sprayberry
+Spraggins
+Spitler
+Spiers
+Sohn
+Seagraves
+Schiffman
+Rudnick
+Rizo
+Riccio
+Rennie
+Quinton
+Quackenbush
+Puma
+Plott
+Pearcy
+Parada
+Paiz
+Munford
+Moskowitz
+Mease
+Mcnary
+Mccusker
+Matt
+Lozoya
+Longmire
+Loesch
+Lasky
+Kuhlmann
+Krieg
+Koziol
+Kowalewski
+Konrad
+Kindle
+Jowers
+Jolin
+Jaco
+Hua
+Horgan
+Hine
+Hileman
+Hepner
+Heise
+Heady
+Hawkinson
+Hannigan
+Haberman
+Guilford
+Grimaldi
+Gilles
+Garton
+Gagliano
+Fruge
+Follett
+Fiscus
+Ferretti
+Ebner
+Easterday
+Eanes
+Dirks
+Dimarco
+Depalma
+Deforest
+Dance
+Cruce
+Craighead
+Christner
+Candler
+Cadwell
+Burchell
+Buettner
+Brinton
+Breed
+Brazier
+Brannen
+Brame
+Bova
+Bomar
+Blakeslee
+Belknap
+Bangs
+Balzer
+Athey
+Armes
+Alvis
+Alverson
+Alvardo
+Alter
+Zhao
+Yeung
+Yen
+Wheelock
+Westlund
+Wessels
+Volkman
+Threadgill
+Thelen
+Tandy
+Tague
+Ta
+Symons
+Swinford
+Sturtevant
+Straka
+Stier
+Stagner
+Segarra
+Seawright
+Sack
+Rutan
+Roux
+Ringler
+Riker
+Ramsdell
+Quattlebaum
+Purifoy
+Poulson
+Permenter
+Peloquin
+Pasley
+Pagel
+Osman
+Obannon
+Nygaard
+Nipper
+Newcomer
+Munos
+Motta
+Meadors
+Mcquiston
+Mcniel
+Mcmann
+Mccrae
+Mayne
+Matte
+Martine
+Lucy
+Legault
+Lechner
+Lack
+Kucera
+Krohn
+Kratzer
+Koopman
+Judson
+Jeske
+Horrocks
+Homes
+Hock
+Hibbler
+Hesson
+Hersh
+Harvin
+Halvorsen
+Griner
+Grindle
+Glen
+Gladstone
+Garofalo
+Frampton
+Forbis
+Fernando
+Eddington
+Diorio
+Dingus
+Dewar
+Desalvo
+Curcio
+Creasy
+Cortese
+Cordoba
+Connally
+Cluff
+Cascio
+Capuano
+Canaday
+Calabro
+Bussard
+Brayton
+Borja
+Bigley
+Arnone
+Arguelles
+Acuff
+Zamarripa
+Wooton
+Wolfgang
+Widner
+Wideman
+Threatt
+Thiele
+Templin
+Teeters
+Synder
+Swint
+Swick
+Sturges
+Stogner
+Stedman
+Spratt
+Six
+Siegfried
+Shetler
+Scull
+Savino
+Sather
+Rothwell
+Rook
+Rone
+Rolf
+Rhee
+Quevedo
+Privett
+Pouliot
+Poche
+Pickel
+Petrillo
+Pellegrini
+Peaslee
+Partlow
+Otey
+Nunnery
+Morelock
+Morello
+Meunier
+Messinger
+Mckie
+Mccubbin
+Mccarron
+Maria
+Lerch
+Lavine
+Laverty
+Lariviere
+Lamkin
+Kugler
+Krol
+Kissel
+Keeter
+Hummer
+Hubble
+Hickox
+Hetzel
+Hayner
+Hagy
+Hadlock
+Groh
+Gregorio
+Gottschalk
+Goodsell
+Gloria
+Gerry
+Gassaway
+Garrard
+Galligan
+Fye
+Firth
+Fenderson
+Feinstein
+Etienne
+Engleman
+Emrick
+Ellender
+Drews
+Doiron
+Degraw
+Deegan
+Dart
+Crissman
+Corr
+Cookson
+Coil
+Cleaves
+Charest
+Chapple
+Chaparro
+Castano
+Carpio
+Byer
+Bufford
+Bridgewater
+Bridgers
+Brandes
+Borrero
+Bonanno
+Aube
+Ancheta
+Abarca
+Abad
+Yung
+Yim
+Wooster
+Woodrow
+Wimbush
+Willhite
+Willams
+Wigley
+Weisberg
+Wardlaw
+Vigue
+Vanhook
+Unknow
+Torre
+Tasker
+Tarbox
+Strachan
+Standard
+Slover
+Shamblin
+Semple
+Schuyler
+Schrimsher
+Sayer
+Salzman
+Salomon
+Rubalcava
+Riles
+Rickey
+Reneau
+Reichel
+Rayfield
+Rabon
+Pyatt
+Prindle
+Poss
+Polito
+Plemmons
+Pesce
+Perrault
+Pereyra
+Ostrowski
+Nilsen
+Niemeyer
+Nick
+Munsey
+Mundell
+Moncada
+Miceli
+Meader
+Mcmasters
+Mckeehan
+Matsumoto
+Marron
+Marden
+Lizarraga
+Lingenfelter
+Lewallen
+Laurence
+Langan
+Lamanna
+Kovac
+Kinsler
+Kephart
+Keown
+Kass
+Kammerer
+Jeffreys
+Hysell
+Householder
+Hosmer
+Hardnett
+Hanner
+Guyette
+Greening
+Glazer
+Ginder
+Fromm
+Fortuna
+Fluellen
+Finkle
+Fey
+Fessler
+Essary
+Eisele
+Duren
+Dittmer
+Crochet
+Cosentino
+Cogan
+Coelho
+Cavin
+Carrizales
+Campuzano
+Brough
+Bow
+Bopp
+Bookman
+Bobb
+Blouin
+Beesley
+Battista
+Bascom
+Bakken
+Badgett
+Arneson
+Anselmo
+Albino
+Ahumada
+Agustin
+Woodyard
+Wolters
+Wireman
+Wilton
+Willison
+Warman
+Wan
+Waldrup
+Vowell
+Vantassel
+Vale
+Twombly
+Toomer
+Tennison
+Teets
+Tedeschi
+Swanner
+Swallow
+Stutz
+Stelly
+Sheehy
+Schermerhorn
+Scala
+Sandidge
+Salters
+Salo
+Saechao
+Roseboro
+Rolle
+Ressler
+Renz
+Renn
+Redford
+Raposa
+Rainbolt
+Pompey
+Pelfrey
+Orndorff
+Oney
+Nolin
+Nimmons
+Ney
+Nardone
+Myhre
+Morman
+Mines
+Menjivar
+Mcglone
+Mccammon
+Maxon
+Maris
+Marciano
+Manus
+Maiden
+Lowrance
+Lorenzen
+Lonergan
+Lollis
+Littles
+Lindahl
+Lansing
+Lamas
+Lach
+Kuster
+Krawczyk
+Knuth
+Knecht
+Kirkendall
+Keitt
+Keever
+Kantor
+Jarboe
+Hoye
+Houchens
+Holter
+Holsinger
+Hickok
+Herb
+Helwig
+Helgeson
+Heater
+Hassett
+Harner
+Hamman
+Hames
+Hadfield
+Goree
+Goldfarb
+Gaughan
+Gaudreau
+Gantz
+Gallion
+Frady
+Foti
+Flesher
+Ferrin
+Faught
+Engram
+Elbert
+Donegan
+Desouza
+Degroot
+Cutright
+Crowl
+Criner
+Coke
+Coan
+Clinkscales
+Chewning
+Chavira
+Catchings
+Carlock
+Bye
+Bulger
+Buenrostro
+Bramblett
+Brack
+Boulware
+Bordeaux
+Bookout
+Bitner
+Birt
+Baranowski
+Baisden
+Augustin
+Allmon
+Alberto
+Acklin
+Yoakum
+Wilbourn
+Whisler
+Weinberger
+Washer
+Vasques
+Vanzandt
+Vanatta
+Troxler
+Tomes
+Tindle
+Tims
+Throckmorton
+Thach
+Stpeter
+Stlaurent
+Stenson
+Spry
+Spitz
+Songer
+Snavely
+Sly
+Sleeper
+Shroyer
+Shortridge
+Shenk
+Sevier
+Seabrook
+Scrivner
+Saltzman
+Rosenberry
+Rockwood
+Robeson
+Roan
+Reiser
+Redwine
+Ramires
+Raber
+Profit
+Posner
+Popham
+Pipes
+Piotrowski
+Pinard
+Peterkin
+Pelham
+Peiffer
+Peay
+Peavey
+Nadler
+Musso
+Milo
+Millett
+Mestas
+Mcgowen
+Marques
+Marasco
+Manriquez
+Manos
+Mair
+Lipps
+Lesser
+Leiker
+Leeds
+Krumm
+Knorr
+Kinslow
+Kessel
+Kendricks
+Kelm
+Ito
+Irick
+Ickes
+Hurlburt
+Horta
+Hoekstra
+Heuer
+Helmuth
+Heatherly
+Hampson
+Hagar
+Haga
+Greenlaw
+Grau
+Godbey
+Gingras
+Gillies
+Gibb
+Gayden
+Gauvin
+Garrow
+Fontanez
+Florio
+Fleischman
+Finke
+Fasano
+Fan
+Faith
+Ezzell
+Ewers
+Eveland
+Eckenrode
+Duclos
+Drumm
+Dimmick
+Delancey
+Defazio
+Deacon
+Dashiell
+Damian
+Cusack
+Crowther
+Crigger
+Cray
+Coolidge
+Coldiron
+Cleland
+Chalfant
+Cassel
+Cape
+Camire
+Cabrales
+Broomfield
+Brittingham
+Brisson
+Brickey
+Braziel
+Brazell
+Bragdon
+Boulanger
+Bos
+Boman
+Bohannan
+Beem
+Barto
+Barre
+Barley
+Baptist
+Azar
+Ashbaugh
+Armistead
+Almazan
+Adamski
+Zendejas
+Winburn
+Willaims
+Wilhoit
+Westberry
+Wentzel
+Wendling
+Wager
+Visser
+Vanscoy
+Vankirk
+Vallee
+Tweedy
+Thornberry
+Sweeny
+Stalker
+Spradling
+Spano
+Smelser
+Shim
+Sechrist
+Schall
+Scaife
+Rugg
+Ruben
+Rothrock
+Roesler
+Riehl
+Ridings
+Render
+Ransdell
+Radke
+Pinero
+Petree
+Pendergast
+Peluso
+Pecoraro
+Pascoe
+Panek
+Oshiro
+Noon
+Navarrette
+Murguia
+Moores
+Moberg
+Mike
+Michaelis
+Mcwhirter
+Mcsweeney
+Mcquade
+Mccay
+Mauk
+Mariani
+Marceau
+Mandeville
+Maeda
+Lunde
+Ludlow
+Loeb
+Lindo
+Linderman
+Leveille
+Leith
+Larock
+Lambrecht
+Kulp
+Kinsley
+Kimberlin
+Kesterson
+Jacinto
+Ice
+Hui
+Hoyos
+Helfrich
+Hanke
+Hail
+Guillermo
+Grisby
+Goyette
+Gouveia
+Glazier
+Gile
+Gerena
+Gelinas
+Gasaway
+Garden
+Funches
+Fujimoto
+Flynt
+Fenske
+Fellers
+Fehr
+Eslinger
+Escalera
+Enciso
+Duley
+Dittman
+Dineen
+Diller
+Devault
+Dao
+Collings
+Clymer
+Clowers
+Chavers
+Charland
+Castorena
+Castello
+Camargo
+Bunce
+Bullen
+Boyes
+Borchers
+Borchardt
+Birnbaum
+Birdsall
+Billman
+Benites
+Bankhead
+Ange
+Ammerman
+Adkison
+Yuan
+Winegar
+Wickman
+Wear
+Warr
+Warnke
+Villeneuve
+Veasey
+Vassallo
+Vannatta
+Vadnais
+Twilley
+Truelove
+Towery
+Tomblin
+Tippett
+Theiss
+Talkington
+Talamantes
+Swart
+Swanger
+Streit
+Straw
+Stines
+Stabler
+Spurling
+Sobel
+Sine
+Simmers
+Shippy
+Shiflett
+Shearin
+Sauter
+Sanderlin
+Rusch
+Runkle
+Ruckman
+Rorie
+Roesch
+Roberto
+Richert
+Rehm
+Randel
+Ragin
+Quesenberry
+Puentes
+Plyler
+Plotkin
+Paugh
+Oshaughnessy
+Ohalloran
+Norsworthy
+Niemann
+Nader
+Moorefield
+Mooneyham
+Modica
+Miyamoto
+Mickel
+Mebane
+Mckinnie
+Mazurek
+Mancilla
+Lukas
+Lovins
+Loughlin
+Lotz
+Lindsley
+Liddle
+Levan
+Lederman
+Leclaire
+Lasseter
+Lapoint
+Lamoreaux
+Lafollette
+Kubiak
+Kirtley
+Keffer
+Kaczmarek
+Jennette
+Housman
+Honey
+Hiers
+Hibbert
+Herrod
+Hegarty
+Hathorn
+Harsh
+Greenhaw
+Grafton
+Govea
+Gardener
+Futch
+Furst
+Frisbee
+Fred
+Franko
+Forcier
+Foran
+Flickinger
+Fairfield
+Eure
+Emrich
+Embrey
+Edgington
+Ecklund
+Eckard
+Durante
+Deyo
+Delvecchio
+Deeds
+Dade
+Currey
+Cuff
+Creswell
+Cottrill
+Casavant
+Cartier
+Cargile
+Capel
+Cammack
+Calfee
+Buzzard
+Burse
+Burruss
+Brust
+Brousseau
+Bridwell
+Braaten
+Borkholder
+Bloomquist
+Bjork
+Bartelt
+Arp
+Amburgey
+Yeary
+Yao
+Whitefield
+Vinyard
+Vicente
+Vanvalkenburg
+Twitchell
+Timmins
+Tester
+Tapper
+Stringham
+Starcher
+Spotts
+Slaugh
+Simonsen
+Sheffer
+Sequeira
+Rosati
+Rode
+Rhymes
+Reza
+Record
+Quint
+Pollak
+Peirce
+Patillo
+Parkerson
+Paiva
+Nilson
+Nice
+Nevin
+Narcisse
+Nair
+Mitton
+Merriam
+Merced
+Meiners
+Mckain
+Mcelveen
+Mcbeth
+Marsden
+Marez
+Manke
+Mahurin
+Mabrey
+Luper
+Krull
+Kees
+Iles
+Hunsicker
+Hornbuckle
+Holtzclaw
+Hirt
+Hinnant
+Heston
+Hering
+Hemenway
+Hegwood
+Hearns
+Halterman
+Halls
+Guiterrez
+Grote
+Granillo
+Grainger
+Glasco
+Gilder
+Garren
+Garlock
+Garey
+Fu
+Fryar
+Fredricks
+Fraizer
+Foxx
+Foshee
+Ferrel
+Felty
+Feathers
+Everitt
+Evens
+Esser
+Elkin
+Eberhart
+Durso
+Duguay
+Driskill
+Doster
+Dewall
+Deveau
+Demps
+Demaio
+Delreal
+Deleo
+Delay
+Deem
+Darrah
+Cumberbatch
+Culberson
+Cranmer
+Cordle
+Colgan
+Chesley
+Cavallo
+Castellon
+Castelli
+Carreras
+Carnell
+Carmon
+Carmen
+Carlucci
+Bottom
+Bontrager
+Blumberg
+Blasingame
+Becton
+Ayon
+Artrip
+Arline
+Andujar
+Alkire
+Alder
+Agan
+Zukowski
+Zuckerman
+Zehr
+Wroblewski
+Wrigley
+Woodside
+Wigginton
+Westman
+Westgate
+Werts
+Washam
+Wardlow
+Walser
+Waiters
+Teller
+Tadlock
+Stuck
+Stringfield
+Stimpson
+Stickley
+Starbuck
+Standish
+Spurlin
+Spindler
+Speller
+Spaeth
+Sotomayor
+Sok
+Sluder
+Shryock
+Shepardson
+Shatley
+Scannell
+Santistevan
+Rosner
+Rolland
+Rhode
+Resto
+Reinhard
+Rathburn
+Prisco
+Poulsen
+Pinney
+Phares
+Pennock
+Pastrana
+Oviedo
+Ostler
+Noto
+Nauman
+Mulford
+Moise
+Moberly
+Mirabal
+Ming
+Metoyer
+Metheny
+Mentzer
+Meldrum
+Mcinturff
+Mcelyea
+Mcdougle
+Massaro
+Lumpkins
+Loveday
+Lofgren
+Loe
+Lirette
+Lesperance
+Lefkowitz
+Ledger
+Lauzon
+Lain
+Lachapelle
+Kurz
+Klassen
+Keough
+Kempton
+Kaelin
+Jeffords
+Im
+Huot
+Hsieh
+Hoyer
+Horwitz
+Hopp
+Hoeft
+Hennig
+Haskin
+Grill
+Gourdine
+Golightly
+Girouard
+Fulgham
+Fritsch
+Freer
+Frasher
+Foulk
+Firestone
+Fiorentino
+Fedor
+Feather
+Ensley
+Englehart
+Eells
+Ebel
+Dunphy
+Donahoe
+Dimas
+Dileo
+Dibenedetto
+Dabrowski
+Crick
+Coonrod
+Conder
+Coddington
+Chunn
+Choy
+Chaput
+Cerna
+Carreiro
+Calahan
+Braggs
+Bourdon
+Boner
+Bollman
+Bittle
+Ben
+Behm
+Bauder
+Batt
+Barreras
+Aubuchon
+Anzalone
+Adamo
+Zhou
+Zerbe
+Zachery
+Witty
+Wirt
+Willcox
+Westberg
+Weikel
+Waymire
+Vroman
+Vinci
+Vallejos
+Tutor
+Truesdell
+Troutt
+Trotta
+Tollison
+Toles
+Tichenor
+Tai
+Symonds
+Surles
+Sunday
+Strayer
+Stgeorge
+Sroka
+Sorrentino
+Solares
+Snelson
+Silvestri
+Sikorski
+Shawver
+Schumaker
+Schorr
+Schooley
+Scates
+Satterlee
+Satchell
+Sacks
+Rymer
+Roselli
+Robitaille
+Riegel
+Richer
+Regis
+Reames
+Provenzano
+Proper
+Priestley
+Plaisance
+Pettey
+Palomares
+Oman
+Nowakowski
+Nace
+Monette
+Minyard
+Mclamb
+Mchone
+Mccarroll
+Masson
+Marco
+Magoon
+Maddy
+Lundin
+Loza
+Licata
+Lesley
+Leonhardt
+Lema
+Landwehr
+Kircher
+Kinch
+Karpinski
+Johannsen
+Hussain
+Houghtaling
+Hoskinson
+Hollaway
+Holeman
+Hobgood
+Hilt
+Hiebert
+Gros
+Gram
+Goggin
+Gentle
+Geissler
+Gadbois
+Gabaldon
+Fleshman
+Flannigan
+Files
+Fairman
+Epp
+Eilers
+Dycus
+Dunmire
+Duffield
+Dowler
+Ditto
+Deloatch
+Dehaan
+Deemer
+Corner
+Clayborn
+Christofferso
+Chilson
+Chesney
+Chatfield
+Charlie
+Caster
+Carron
+Canale
+Camden
+Buff
+Brigman
+Branstetter
+Bosse
+Borton
+Bonar
+Blau
+Biron
+Beagle
+Barroso
+Arvin
+Arispe
+Zacharias
+Zabel
+Yaeger
+Works
+Woolford
+Whetzel
+Weakley
+Veatch
+Vandeusen
+Tufts
+Troxel
+Troche
+Traver
+Townsel
+Tosh
+Talarico
+Swilley
+Sterrett
+Stenger
+Springfield
+Speakman
+Sowards
+Sours
+Souders
+Souder
+Soles
+Sobers
+Snoddy
+Smither
+Sias
+Shute
+Shoaf
+Shahan
+Schuetz
+Scaggs
+Santini
+Rosson
+Rolen
+Robidoux
+Rentas
+Recio
+Pixley
+Pawlowski
+Pawlak
+Paull
+Pascal
+Overbey
+Orear
+Oliveri
+Oldenburg
+Nutting
+Naugle
+Mote
+Mossman
+Moor
+Misner
+Milazzo
+Michelson
+Mei
+Mcentee
+Mccullar
+Mccree
+Mcaleer
+Mazzone
+Maxim
+Marshal
+Mandell
+Manahan
+Malott
+Maisonet
+Mailloux
+Lumley
+Lowrie
+Louviere
+Lipinski
+Lindemann
+Leppert
+Leopold
+Leasure
+Leaf
+Labarge
+Kubik
+Knisely
+Knepp
+Kenworthy
+Kennelly
+Kelch
+Karg
+Kanter
+Ignacio
+Hyer
+Houchin
+Hosley
+Hosler
+Hollon
+Holleman
+Heitman
+Hebb
+Haggins
+Gwaltney
+Guin
+Greenman
+Goulding
+Gorden
+Goodyear
+Geraci
+Georges
+Gathers
+Frison
+Feagin
+Falconer
+Espada
+Erving
+Erikson
+Eisenhauer
+Eder
+Ebeling
+Durgin
+Drown
+Dowdle
+Dinwiddie
+Delcastillo
+Dedrick
+Crimmins
+Covell
+Cournoyer
+Coria
+Cohan
+Cataldo
+Carpentier
+Canas
+Campa
+Brode
+Brashears
+Blaser
+Bicknell
+Berk
+Bednar
+Barwick
+Ascencio
+Althoff
+Almodovar
+Alamo
+Zirkle
+Zabala
+Xu
+Wolverton
+Winebrenner
+Wetherell
+Westlake
+Wegener
+Weddington
+Vong
+Tuten
+Trosclair
+Trim
+Tressler
+Theroux
+Teske
+Sword
+Swinehart
+Swensen
+Sundquist
+Southall
+Socha
+Sizer
+Silverberg
+Shortt
+Shimizu
+Sherrard
+Shen
+Shaeffer
+Seth
+Scheid
+Scheetz
+Saravia
+Sanner
+Rubinstein
+Rozell
+Romer
+Ringo
+Rheaume
+Reisinger
+Raven
+Randles
+Pullum
+Petrella
+Payan
+Papp
+Pablo
+Nordin
+Norcross
+Nicoletti
+Nicholes
+Newbold
+Nakagawa
+Mraz
+Monteith
+Milstead
+Milliner
+Mellen
+Mccardle
+Matthias
+Marcy
+Luft
+Loo
+Locker
+Liptak
+Lipp
+Leitch
+Latimore
+Larrison
+Landau
+Laborde
+Koval
+Izquierdo
+Hymel
+Hoskin
+Holte
+Hoefer
+Hayworth
+Hausman
+Harrill
+Harrel
+Hardt
+Gully
+Groover
+Grinnell
+Greenspan
+Graver
+Grandberry
+Gorrell
+Goldenberg
+Goguen
+Gilleland
+Garr
+Fuson
+Foye
+Felt
+Feldmann
+Everly
+Dyess
+Dyal
+Dunnigan
+Downie
+Dolby
+Divine
+Deatherage
+Dates
+Danna
+Cosey
+Corrado
+Cheever
+Celaya
+Caver
+Cashion
+Caplinger
+Cansler
+Byrge
+Bruder
+Brew
+Breuer
+Breslin
+Brazelton
+Botkin
+Bonneau
+Bones
+Bondurant
+Bohanan
+Bogue
+Boes
+Bodner
+Boatner
+Blatt
+Bickley
+Belliveau
+Beiler
+Beier
+Beckstead
+Bart
+Bang
+Bachmann
+Atkin
+Aron
+Andreas
+Altizer
+Alloway
+Allaire
+Albro
+Abron
+Zellmer
+Yetter
+Yelverton
+Wiltshire
+Wiens
+Whidden
+Wait
+Viramontes
+Vanwormer
+Topper
+Tarantino
+Tanksley
+Sumlin
+Strauch
+Strang
+Stice
+Spahn
+Sosebee
+Sigala
+Shrout
+Seamon
+Schrum
+Schneck
+Schantz
+Said
+Ruddy
+Romig
+Roehl
+Renninger
+Reding
+Pyne
+Polak
+Pohlman
+Pasillas
+Oldfield
+Oldaker
+Ohanlon
+Ogilvie
+Norberg
+Nolette
+Nies
+Neufeld
+Nellis
+Mummert
+Mulvihill
+Mullaney
+Monteleone
+Mendonca
+Meisner
+Mcmullan
+Mccluney
+Mattis
+Massengill
+Manfredi
+Luedtke
+Lounsbury
+Lora
+Liberatore
+Leek
+Lease
+Lazaro
+Lamphere
+Laforge
+Kuo
+Koo
+Jourdan
+Ismail
+Iorio
+Iniguez
+Ikeda
+Hubler
+Hodgdon
+Hocking
+Heacock
+Haslam
+Haralson
+Hanshaw
+Hannum
+Hallam
+Haden
+Garnes
+Garces
+Gammage
+Gambino
+Finkel
+Faucett
+Fahy
+Esteban
+Ehrhardt
+Eggen
+Dusek
+Durrant
+Dubay
+Dones
+Dey
+Depasquale
+Delucia
+Degraff
+Deer
+Decamp
+Davalos
+Darwin
+Dan
+Cullins
+Conard
+Clouser
+Clontz
+Cifuentes
+Chico
+Chappel
+Chaffins
+Celis
+Carwile
+Byram
+Bruggeman
+Brick
+Bressler
+Brathwaite
+Brasfield
+Bradburn
+Boose
+Boon
+Bodie
+Blosser
+Blas
+Bise
+Bertsch
+Bernardi
+Bernabe
+Bengtson
+Barrette
+Astorga
+Armand
+Antone
+Alday
+Albee
+Abrahamson
+Yarnell
+Wiltse
+Wile
+Wiebe
+Waguespack
+Vasser
+Upham
+Tyre
+Turek
+Tune
+Traxler
+Torain
+Tomaszewski
+Tinnin
+Tiner
+Tindell
+Teed
+Styron
+Stahlman
+Staab
+Spoon
+Spells
+Skiba
+Shih
+Sheperd
+Seidl
+Secor
+Schutte
+Sanfilippo
+Ruder
+Rondon
+Reina
+Rearick
+Rank
+Procter
+Prochaska
+Pettengill
+Pauly
+Neilsen
+Nally
+Mutter
+Mullenax
+Morano
+Meads
+Mcnaughton
+Mcmurtry
+Mcmath
+Mckinsey
+Matthes
+Massenburg
+Marlar
+Margolis
+Marcos
+Malin
+Magallon
+Mackin
+Lovette
+Loughran
+Loring
+Longstreet
+Loiselle
+Lenihan
+Laub
+Kunze
+Kull
+Koepke
+Knights
+Kerwin
+Kalinowski
+Kagan
+Innis
+Innes
+Husband
+Holtzman
+Heinemann
+Harshman
+Haider
+Haack
+Guss
+Grondin
+Grissett
+Greenawalt
+Gravel
+Goudy
+Goodlett
+Goldston
+Gokey
+Goin
+Gardea
+Galaviz
+Gafford
+Gabrielson
+Furlow
+Fritch
+Fordyce
+Folger
+Elizalde
+Ehlert
+Eckhoff
+Eccleston
+Ealey
+Dubin
+Dolphin
+Dieter
+Diemer
+Deschamps
+Delapena
+Decicco
+Debolt
+Daum
+Cullinan
+Crittendon
+Crase
+Cossey
+Coppock
+Coots
+Colyer
+Columbus
+Cluck
+Chamberland
+Cane
+Burkhead
+Bumpus
+Buchan
+Borman
+Bork
+Boe
+Birkholz
+Berardi
+Benda
+Behnke
+Barter
+Auer
+Amezquita
+Wotring
+Wirtz
+Wingert
+Wiesner
+Whitesides
+Weyant
+Wainscott
+Vivian
+Venezia
+Varnell
+Tussey
+Trainer
+Toll
+Thurlow
+Tack
+Tabares
+Stiver
+Stell
+Starke
+Stanhope
+Stanek
+Sisler
+Sinnott
+Sidney
+Siciliano
+Shehan
+Selph
+Seager
+Scurlock
+Scranton
+Santucci
+Santangelo
+Saltsman
+Ruel
+Ropp
+Rolling
+Rogge
+Rettig
+Renwick
+Reidy
+Reider
+Redfield
+Quam
+Premo
+Port
+Pier
+Peet
+Parente
+Paolucci
+Pan
+Palmquist
+Orme
+Ohler
+Ogg
+Netherton
+Mutchler
+Morita
+Mistretta
+Minnis
+Middendorf
+Menzel
+Mendosa
+Mendelson
+Meaux
+Mcspadden
+Mcquaid
+Mcnatt
+Manigault
+Maney
+Mager
+Lung
+Lukes
+Lopresti
+Liriano
+Lipton
+Letson
+Lechuga
+Lazenby
+Lauria
+Larimore
+Kwok
+Kwak
+Krupp
+Krupa
+Krum
+Kopec
+Kinchen
+Kifer
+Kerney
+Kerner
+Kennison
+Kegley
+Kays
+Karcher
+Justis
+Johson
+Jellison
+Janke
+Isabell
+Huskins
+Holzman
+Hollie
+Hinojos
+Highland
+Hefley
+He
+Hatmaker
+Harte
+Halloway
+Hallenbeck
+Goodwyn
+Glaspie
+Gillian
+Geise
+Fullwood
+Fryman
+Frew
+Frakes
+Fraire
+Farrer
+Enlow
+Engen
+Ellzey
+Eckles
+Earles
+Ealy
+Dunkley
+Drinkard
+Dreiling
+Draeger
+Dinardo
+Dills
+Desroches
+Desantiago
+Current
+Curlee
+Crumbley
+Critchlow
+Coury
+Courtright
+Coffield
+Cleek
+Christen
+Charpentier
+Cardone
+Caples
+Cantin
+Buntin
+Bugbee
+Brinkerhoff
+Brackin
+Bourland
+Bohl
+Bogdan
+Blassingame
+Beacham
+Banning
+Auguste
+Andreasen
+Amann
+Almon
+Alejo
+Adelman
+Abston
+Zeno
+Yerger
+Wymer
+Woodberry
+Windley
+Whiteaker
+Westfield
+Weibel
+Wanner
+Waldrep
+Vital
+Villani
+Vanarsdale
+Utterback
+Updike
+Triggs
+Topete
+Tolar
+Tigner
+Thoms
+Tauber
+Tarvin
+Tally
+Swiney
+Sweatman
+Studebaker
+Streets
+Stennett
+States
+Starrett
+Stannard
+Stalvey
+Sonnenberg
+Smithey
+Sieber
+Sickles
+Shinault
+Segars
+Sanger
+Salmeron
+Rothe
+Rizzi
+Rine
+Ricard
+Restrepo
+Ralls
+Ragusa
+Quiroga
+Ping
+Phung
+Pero
+Pegg
+Pavlik
+Papenfuss
+Oropeza
+Omar
+Okane
+Neer
+Nee
+Nathaniel
+Mudge
+Mozingo
+Molinaro
+Mikel
+Mcvicker
+Mcgarvey
+Mcfalls
+Mccraney
+Matus
+Magers
+Llanos
+Livermore
+Liss
+Linehan
+Leto
+Leitner
+Laymon
+Lawing
+Lawerence
+Lacourse
+Kwong
+Kollar
+Kneeland
+Keo
+Kennett
+Kellett
+Kangas
+Janzen
+Hutter
+Huse
+Huling
+Hoss
+Hohn
+Hofmeister
+Hewes
+Hern
+Harjo
+Habib
+Gust
+Guice
+Grullon
+Greggs
+Grayer
+Granier
+Grable
+Gowdy
+Giannini
+Getchell
+Gartman
+Garnica
+Ganey
+Gallimore
+Fray
+Fetters
+Fergerson
+Farlow
+Fagundes
+Exley
+Esteves
+Enders
+Edenfield
+Easterwood
+Drakeford
+Dipasquale
+Desousa
+Deshields
+Deeter
+Dedmon
+Debord
+Daughtery
+Cutts
+Courtemanche
+Coursey
+Copple
+Coomes
+Collis
+Coll
+Cogburn
+Clopton
+Choquette
+Chaidez
+Castrejon
+Calhoon
+Burbach
+Bulloch
+Buchman
+Bruhn
+Bohon
+Blough
+Bien
+Belmont
+Baynes
+Barstow
+Zeman
+Zackery
+Yardley
+Yamashita
+Wulff
+Wilken
+Wiliams
+Wickersham
+Wible
+Whipkey
+Wedgeworth
+Walmsley
+Walkup
+Vreeland
+Verrill
+Valera
+Umana
+Traub
+Timothy
+Swingle
+Swing
+Summey
+Stroupe
+Stockstill
+Steffey
+Stefanski
+Statler
+Stapp
+Speights
+Sons
+Solari
+Soderberg
+Slick
+Shunk
+Shorey
+Shewmaker
+Sheilds
+Schiffer
+Schank
+Schaff
+Sagers
+Rodger
+Rochon
+Riser
+Rickett
+Reale
+Raglin
+Poon
+Polly
+Polen
+Plata
+Pitcock
+Percival
+Palen
+Pahl
+Orona
+Oberle
+Nocera
+Navas
+Nault
+Mullings
+Mouser
+Moos
+Montejano
+Monreal
+Minick
+Middlebrook
+Meece
+Mcmillion
+Mccullen
+Mauck
+Marshburn
+Maillet
+Mahaney
+Magner
+Maclin
+Lucey
+Litteral
+Lippincott
+Leite
+Leis
+Leaks
+Laurie
+Lamarre
+Kost
+Jurgens
+Jesus
+Jerkins
+Jager
+Hurwitz
+Hughley
+Hotaling
+Horstman
+Hohman
+Hocker
+Hively
+Hipps
+Hile
+Hessler
+Hermanson
+Hepworth
+Henn
+Helland
+Hedlund
+Harkless
+Haigler
+Gutierez
+Gum
+Grindstaff
+Glantz
+Giardina
+Gerken
+Gadsden
+Freda
+Finnerty
+Feld
+Farnum
+Encinas
+Elton
+Eager
+Drakes
+Dennie
+Cutlip
+Curtsinger
+Couto
+Cortinas
+Corby
+Choice
+Chiasson
+Carle
+Carballo
+Brindle
+Borum
+Bober
+Blagg
+Birk
+Berthiaume
+Beahm
+Batres
+Basnight
+Barbara
+Backes
+Axtell
+Aust
+Au
+Atterberry
+Alvares
+Alt
+Alegria
+Abe
+Yow
+Yip
+Woodell
+Wojciechowski
+Winfree
+Winbush
+Wiest
+Wesner
+Wax
+Wamsley
+Wakeman
+Verner
+Truex
+Trafton
+Toman
+Thorsen
+Thor
+Theus
+Tellier
+Tallant
+Szeto
+Strope
+Stills
+Stage
+Sorg
+Simkins
+Shuey
+Shaul
+Servin
+Serio
+Serafin
+Senior
+Sebring
+Salguero
+Saba
+Ryerson
+Rudder
+Ruark
+Rother
+Rohrbaugh
+Rohrbach
+Rohan
+Rogerson
+Risher
+Rigg
+Reeser
+Pryce
+Prokop
+Prins
+Priebe
+Prejean
+Pinheiro
+Petrone
+Petri
+Penson
+Pearlman
+Parikh
+Pal
+Pair
+Natoli
+Murakami
+Mullikin
+Mullane
+Motes
+Morningstar
+Monks
+Mcveigh
+Mcgrady
+Mcgaughey
+Mccurley
+Masi
+Marchan
+Manske
+Maine
+Maez
+Lusby
+Linde
+Lile
+Likens
+Licon
+Leroux
+Lemaire
+Legette
+Lax
+Laskey
+Laprade
+Laplant
+Lady
+Kolar
+Kittredge
+Kinley
+Kerber
+Kanagy
+Johannes
+Jetton
+Jayne
+January
+Janik
+Ippolito
+Inouye
+Hunsinger
+Howley
+Howery
+Horrell
+Hoosier
+Holthaus
+Hiner
+Hilson
+Hilderbrand
+Hasan
+Hartzler
+Harnish
+Harada
+Hansford
+Halligan
+Hagedorn
+Gwynn
+Gudino
+Greenstein
+Greear
+Gracey
+Goudeau
+Gose
+Goodner
+Ginsburg
+Gerth
+Gerner
+Fyfe
+Fujii
+Frier
+Frenette
+Folmar
+Fleisher
+Fleischmann
+Fetzer
+Fern
+Eisenman
+Earhart
+Dupuy
+Dunkelberger
+Drummer
+Drexler
+Dillinger
+Dilbeck
+Diana
+Dewald
+Demby
+Deford
+Daniell
+Dake
+Craine
+Como
+Clever
+Chesnut
+Casady
+Carstens
+Carrick
+Carino
+Carignan
+Canchola
+Cale
+Bushong
+Burman
+Buono
+Brownlow
+Broach
+Britten
+Brickhouse
+Boyden
+Boulton
+Borne
+Borland
+Bohrer
+Blubaugh
+Bever
+Berggren
+Benevides
+Arocho
+Arends
+Amezcua
+Almendarez
+Zalewski
+Witzel
+Winkfield
+Wilhoite
+Vara
+Vangundy
+Vanfleet
+Vanetten
+Vandergriff
+Urbanski
+Tyrell
+Troiano
+Tickle
+Thibodaux
+Straus
+Stoneking
+Stjean
+Stillings
+Stiff
+Stange
+Square
+Speicher
+Speegle
+Sowa
+Smeltzer
+Slawson
+Simmonds
+Shuttleworth
+Serpa
+Senger
+Seidman
+Schweiger
+Schloss
+Schimmel
+Schechter
+Sayler
+Sabb
+Sabatini
+Ronan
+Rodiguez
+Riggleman
+Richins
+Reep
+Reamer
+Prunty
+Porath
+Plunk
+Piland
+Philbrook
+Pettitt
+Perna
+Peralez
+Pascale
+Padula
+Oboyle
+Nivens
+Nickols
+Murph
+Mundt
+Munden
+Montijo
+Mcmanis
+Mcgrane
+Mccrimmon
+Manzi
+Mangold
+Malick
+Mahar
+Maddock
+Lust
+Losey
+Loop
+Litten
+Liner
+Leff
+Leedy
+Leavell
+Ladue
+Krahn
+Kluge
+Junker
+Iversen
+Imler
+Hurtt
+Huizar
+Hubbert
+Howington
+Hollomon
+Holdren
+Hoisington
+Hise
+Heiden
+Hauge
+Hartigan
+Gutirrez
+Griffie
+Greenhill
+Gratton
+Granata
+Gottfried
+Gertz
+Gautreaux
+Furry
+Furey
+Funderburg
+Flippen
+Fitzgibbon
+Fergus
+Felice
+Eye
+Dyar
+Drucker
+Donoghue
+Dildy
+Devers
+Detweiler
+Despres
+Denby
+Degeorge
+Cueto
+Cranston
+Courville
+Clukey
+Cirillo
+Chon
+Chivers
+Caudillo
+Catt
+Butera
+Bulluck
+Buckmaster
+Braunstein
+Bracamonte
+Bourdeau
+Border
+Bonnette
+Bobadilla
+Boaz
+Blackledge
+Beshears
+Bernhard
+Bergeson
+Baver
+Barthel
+Balsamo
+Bak
+Aziz
+Awad
+Authement
+Altom
+Altieri
+Abels
+Zigler
+Zhu
+Younker
+Yeomans
+Yearwood
+Wurster
+Winget
+Whitsett
+Wechsler
+Weatherwax
+Wathen
+Warriner
+Wanamaker
+Walraven
+Viens
+Vandemark
+Vancamp
+Uchida
+Triana
+Tinoco
+Terpstra
+Tellis
+Tarin
+Taranto
+Takacs
+Studdard
+Struthers
+Strout
+Stiller
+Spataro
+Soderquist
+Sliger
+Silberman
+Shurtleff
+Sheetz
+Schillinger
+Ritch
+Reif
+Raybon
+Ratzlaff
+Radley
+Putt
+Putney
+Prime
+Press
+Pinette
+Piner
+Petrin
+Parise
+Osbourne
+Nyman
+Northington
+Noblitt
+Nishimura
+Nell
+Neher
+Nalls
+Naccarato
+Mucha
+Mounce
+Miron
+Millis
+Meaney
+Mcnichols
+Mckinnis
+Mcjunkin
+Mcduffy
+Max
+Marcello
+Manrique
+Mannion
+Mangual
+Malveaux
+Mains
+Lumsden
+Lucien
+Lohmann
+Lipe
+Lightsey
+Lemasters
+Leist
+Laxton
+Laverriere
+Latorre
+Lamons
+Kral
+Kopf
+Knauer
+Kitt
+Kaul
+Karas
+Kamps
+Jusino
+Janis
+Islam
+Hullinger
+Huges
+Hornung
+Hiser
+Hempel
+Helsel
+Hassinger
+Hargraves
+Hammes
+Hallberg
+Gutman
+Gumbs
+Gruver
+Graddy
+Gonsales
+Goncalves
+Glennon
+Gilford
+Geno
+Freshour
+Flippo
+Fifer
+Few
+Fermin
+Fason
+Farrish
+Fallin
+Ewert
+Estepp
+Escudero
+Ensminger
+Emmanuel
+Emberton
+Elms
+Ellerbe
+Eide
+Dysart
+Dougan
+Dierking
+Dicus
+Detrick
+Deroche
+Depue
+Demartino
+Delosreyes
+Dalke
+Culbreath
+Crownover
+Crisler
+Crass
+Corsi
+Chagnon
+Centers
+Cavanagh
+Casson
+Carollo
+Cadwallader
+Burnley
+Burciaga
+Burchard
+Broadhead
+Boris
+Booze
+Bolte
+Body
+Berens
+Bellman
+Bellard
+Baril
+Arden
+Antonucci
+Amado
+Allie
+Wolfgram
+Winsor
+Wimbish
+Wilbert
+Wier
+Wallach
+Viveros
+Vento
+Varley
+Vanslyke
+Vangorder
+Touchstone
+Tomko
+Tiemann
+Throop
+Tamura
+Talmadge
+Swayze
+Sturdevant
+Strauser
+Stolz
+Stenberg
+Stayton
+Spohn
+Spillers
+Spillane
+Sluss
+Sloane
+Slavens
+Simonetti
+Shofner
+Shead
+Senecal
+Seales
+Schueler
+Schley
+Schacht
+Sauve
+Sarno
+Salsbury
+Rothschild
+Rosier
+Rines
+Reveles
+Rein
+Redus
+Redfern
+Reck
+Ranney
+Raggs
+Prout
+Prill
+Preble
+Prager
+Plemons
+Pippen
+Pilon
+Piccirillo
+Pewitt
+Pesina
+Pecora
+Otani
+Orsini
+Ollie
+Oestreich
+Odea
+Ocallaghan
+Northup
+Niehaus
+Newberg
+Nasser
+Narron
+Monarrez
+Mishler
+Mcsherry
+Mcelfresh
+Mayon
+Mauer
+Mattice
+Mash
+Marrone
+Marmolejo
+Marini
+Marie
+Mara
+Malm
+Machen
+Lunceford
+Loewen
+Liverman
+Litwin
+Linscott
+Levins
+Lenox
+Legaspi
+Leeman
+Leavy
+Lannon
+Lamson
+Lambdin
+Labarre
+Knouse
+Klemm
+Kleinschmidt
+Kirklin
+Keels
+Juliano
+Howser
+Hott
+Hosier
+Hosea
+Hopwood
+Holyfield
+Hodnett
+Hirsh
+Heimann
+Height
+Heckel
+Harger
+Hamil
+Hajek
+Gurganus
+Gunning
+Grange
+Gonzalas
+Goggins
+Gerow
+Gaydos
+Garduno
+Ganley
+Galey
+Farner
+Ester
+Engles
+Emond
+Emert
+Ellenburg
+Edick
+Duell
+Dublin
+Dorazio
+Dong
+Dimond
+Diederich
+Dewalt
+Depuy
+Dempster
+Demaria
+Dehoyos
+Dearth
+Dealba
+Dane
+Czech
+Crose
+Crespin
+Cogdill
+Clinard
+Cipriano
+Chretien
+Chalk
+Cerny
+Ceniceros
+Celestin
+Caple
+Cacho
+Burrill
+Buhr
+Buckland
+Branam
+Boysen
+Bovee
+Boos
+Boler
+Blom
+Blasko
+Beyers
+Belz
+Belmonte
+Bednarz
+Beckmann
+Beaudin
+Bazile
+Barbeau
+Balentine
+Abrahams
+Able
+Zielke
+Yunker
+Yeates
+Wrobel
+Wike
+Whisnant
+Wherry
+Wagnon
+Vogan
+Vansant
+Vannest
+Vallo
+Ullery
+Towles
+Towell
+Tiger
+Thill
+Taormina
+Tannehill
+Taing
+Storrs
+Stickles
+Stetler
+Sparling
+Solt
+Silcox
+Sheard
+Shadle
+Seman
+Selleck
+Schlemmer
+Scher
+Sapien
+Sainz
+Rumble
+Roye
+Rosamond
+Romain
+Rizzuto
+Resch
+Rentz
+Rather
+Rasch
+Ranieri
+Purtell
+Primmer
+Portwood
+Pontius
+Pons
+Pletcher
+Pledger
+Pirkle
+Pillsbury
+Pentecost
+Peng
+Paxson
+Ortez
+Organ
+Oles
+Newborn
+Mullett
+Muirhead
+Mouzon
+Mork
+Mollett
+Mohn
+Mitcham
+Melillo
+Mee
+Medders
+Mcmiller
+Mccleery
+Mccaughey
+Manders
+Mak
+Maciejewski
+Macaulay
+Lute
+Lipman
+Lewter
+Larocque
+Langton
+Kriner
+Knipp
+Killeen
+Karn
+Kalish
+Kaczor
+Jonson
+Jerez
+Jarrard
+Janda
+Hymes
+Hollman
+Hollandsworth
+Holl
+Hobdy
+Hitch
+Hennen
+Hemmer
+Hagins
+Haddox
+Guitierrez
+Guernsey
+Gorsuch
+Gholson
+Genova
+Gazaway
+Gauna
+Gammons
+Freels
+Fonville
+Fly
+Florian
+Fleet
+Fetterman
+Fava
+Farquhar
+Farish
+Fabela
+Escoto
+Eisen
+Dossett
+Dority
+Dorfman
+Demmer
+Dehn
+Dawley
+Darbonne
+Damore
+Damm
+Crosley
+Cron
+Crompton
+Crichton
+Cotner
+Cordon
+Conerly
+Colvard
+Clauson
+Chess
+Cheeseman
+Charity
+Cavallaro
+Castille
+Cabello
+Burgan
+Buffum
+Bruss
+Brassfield
+Bowerman
+Bothwell
+Borgen
+Bonaparte
+Bombard
+Boivin
+Boissonneault
+Bogner
+Bodden
+Boan
+Blanche
+Bittinger
+Bickham
+Bedolla
+Bale
+Bainbridge
+Aybar
+Avendano
+Ashlock
+Amidon
+Almanzar
+Akridge
+Ackermann
+Zager
+Yong
+Xavier
+Worrall
+Winans
+Wilsey
+Wightman
+Westrick
+Wenner
+Warne
+Warford
+Verville
+Utecht
+Upson
+Tuma
+Tseng
+Troncoso
+Trollinger
+Torbert
+Taulbee
+Sutterfield
+Stough
+Storch
+Stonebraker
+Stolle
+Stilson
+Stiefel
+Steptoe
+Stepney
+Stender
+Stemple
+Staggers
+Spurrier
+Spray
+Spinney
+Spengler
+Smartt
+Skoog
+Silvis
+Sieg
+Shuford
+Selfridge
+Seguin
+Sedgwick
+Sease
+Scotti
+Schroer
+Schlenker
+Schill
+Savarese
+Sapienza
+Sanson
+Sandefur
+Salamone
+Rusnak
+Rudisill
+Royalty
+Rothermel
+Roca
+Resendiz
+Reliford
+Rasco
+Raiford
+Quisenberry
+Quijada
+Pullins
+Puccio
+Postell
+Poppe
+Pinter
+Piche
+Petrucci
+Pellegrin
+Pelaez
+Patti
+Paton
+Pasco
+Parkes
+Paden
+Pabst
+Orchard
+Olmsted
+Newlon
+Mynatt
+Mustafa
+Mower
+Morrone
+Moree
+Moffat
+Mixson
+Minner
+Min
+Millette
+Mederos
+Mcgahan
+Mcconville
+Maughan
+Massingill
+Marano
+Macri
+Lovern
+Lichtenstein
+Leonetti
+Lehner
+Lawley
+Laramie
+Lappin
+Lahti
+Lago
+Lacayo
+Kuester
+Knee
+Kincade
+Junior
+Juhl
+Joslyn
+Jiron
+Jessop
+Jerry
+Jarosz
+Jain
+Hults
+Hoge
+Hodgins
+Hoban
+Hinkson
+Hillyard
+Herzig
+Hervey
+Henriksen
+Hawker
+Hause
+Hard
+Hankerson
+Gregson
+Golliday
+Gilcrease
+Gessner
+Gerace
+Garwood
+Garst
+Gaillard
+Flinchum
+Fishel
+Fishback
+Filkins
+Fentress
+Fabre
+Ethier
+Espana
+Eisner
+Ehrhart
+Efird
+Drennon
+Dominy
+Dominique
+Domingue
+Dipaolo
+Dinan
+Dimartino
+Deskins
+Dengler
+Defreitas
+Defranco
+Dancer
+Dahlin
+Cutshaw
+Cuthbert
+Croyle
+Crothers
+Critchfield
+Cowie
+Costner
+Coppedge
+Copes
+Ciccone
+Champ
+Cesar
+Caufield
+Capo
+Cambron
+Cambridge
+Buser
+Burnes
+Buhl
+Buendia
+Brindley
+Brecht
+Bourgoin
+Boomer
+Blackshire
+Birge
+Benninger
+Bembry
+Beil
+Begaye
+Barrentine
+Barks
+Banton
+Balmer
+Baity
+Auerbach
+Ambler
+Alexandre
+Ackerson
+Zurcher
+Zell
+Wynkoop
+Wallick
+Waid
+Vos
+Vizcaino
+Vester
+Veale
+Vandermark
+Vanderford
+Tuthill
+Trivette
+Thiessen
+Tewksbury
+Tao
+Tabron
+Swim
+Swasey
+Swanigan
+Stoughton
+Stoudt
+Stimson
+Stecker
+Stead
+Stall
+Spady
+Souther
+Smoak
+Sklar
+Simcox
+Sidwell
+Sharon
+Seybert
+Sesco
+Seeman
+Seaborn
+Schwenk
+Schmeling
+Rossignol
+Robillard
+Robicheaux
+Riveria
+Rippeon
+Ridgley
+Remaley
+Rehkop
+Reddish
+Reach
+Rauscher
+Rachel
+Quirion
+Pusey
+Pruden
+Pressler
+Potvin
+Pospisil
+Paradiso
+Pangburn
+Palmateer
+Ownby
+Otwell
+Osterberg
+Osmond
+Olsson
+Old
+Oberlander
+Nusbaum
+Novack
+Nokes
+Nicastro
+Nehls
+Nay
+Naber
+Mulhern
+Motter
+Moretz
+Milian
+Mercedes
+Mckeel
+Mcclay
+Mccart
+Matsuda
+Mary
+Martucci
+Marple
+Marko
+Marciniak
+Manes
+Mancia
+Maker
+Macrae
+Lybarger
+Lint
+Lineberger
+Levingston
+Lecroy
+Lattimer
+Laseter
+Kulick
+Krier
+Knutsen
+Klem
+Kinne
+Kinkade
+Ketterman
+Kerstetter
+Kersten
+Karam
+Jury
+Joshi
+Jin
+Jent
+Jefcoat
+Hillier
+Hillhouse
+Hettinger
+Henthorn
+Henline
+Helzer
+Heitzman
+Heineman
+Heenan
+Haughton
+Haris
+Harbert
+Haman
+Grinstead
+Gremillion
+Gorby
+Giraldo
+Gioia
+Gerardi
+Geraghty
+Gaunt
+Gatson
+Gardin
+Gans
+Gammill
+Games
+Gain
+Friedlander
+Frahm
+Fossett
+Fosdick
+Forth
+Forbush
+Fondren
+Fleckenstein
+Fitchett
+Filer
+Feliz
+Feist
+Ewart
+Evelyn
+Esters
+Elsner
+Edgin
+Eddie
+Easterly
+Dussault
+Durazo
+Don
+Devereaux
+Deshotel
+Deckert
+Dargan
+Dare
+Cornman
+Conkle
+Condit
+Commander
+Claunch
+Clabaugh
+Chute
+Cheesman
+Chea
+Charney
+Charleston
+Casella
+Carone
+Carbonell
+Canipe
+Campana
+Calles
+Cabezas
+Cabell
+Buttram
+Bustillos
+Buskirk
+Boyland
+Bourke
+Blakeley
+Big
+Berumen
+Berrier
+Bench
+Belli
+Behrendt
+Baumbach
+Bartsch
+Baney
+Arambula
+Alldredge
+Allbritton
+Ziemba
+Zanders
+Youngquist
+Yoshioka
+Yohe
+Wunder
+Woodfin
+Wojtowicz
+Winkel
+Wilmore
+Willbanks
+Wesolowski
+Wendland
+Walko
+Votaw
+Vanek
+Uriarte
+Urbano
+Turnipseed
+Triche
+Trautman
+Towler
+Tokarz
+Temples
+Tefft
+Teegarden
+Syed
+Swigart
+Stryker
+Stoller
+Stapler
+Stansfield
+Smit
+Smelley
+Sicard
+Shulman
+Shew
+Shear
+Sheahan
+Sharpton
+Selvidge
+Schlesinger
+Savell
+Sandford
+Sabatino
+Rosenbloom
+Roepke
+Rish
+Rhames
+Renken
+Reger
+Rappaport
+Quarterman
+Puig
+Prasad
+Poplar
+Pizano
+Pigott
+Pick
+Phair
+Petrick
+Patt
+Pascua
+Paramore
+Papineau
+Olivieri
+Ogren
+Norden
+Noga
+Nisbet
+Munk
+Munch
+Mui
+Morvant
+Moro
+Moloney
+Merz
+Meng
+Meltzer
+Mellinger
+Mehl
+Mcnealy
+Mckernan
+Mchaney
+Mccleskey
+Mcandrews
+Mayton
+Mayor
+Markert
+Maresca
+Marcellus
+Maner
+Mandujano
+Malpass
+Macintyre
+Lytton
+Lyall
+Lummus
+Longshore
+Longfellow
+Lokey
+Locher
+Leverette
+Lepe
+Lefever
+Leeson
+Lederer
+Lampert
+Lagrone
+La
+Kreider
+Korth
+Knopf
+Kleist
+Kiss
+Keltner
+Kelling
+Kaspar
+Kappler
+Justin
+Josephs
+Jiang
+Huckins
+Horace
+Holub
+Hofstetter
+Hoehn
+Higginson
+Hennings
+Heid
+Havel
+Hauer
+Harnden
+Hargreaves
+Hanger
+Guild
+Guidi
+Grate
+Grandy
+Grandstaff
+Goza
+Goodridge
+Goodfellow
+Goggans
+Godley
+Giusti
+Gilyard
+Geoghegan
+Galyon
+Gaeta
+Funes
+Font
+Flor
+Flanary
+Fales
+Erlandson
+Ellett
+Elia
+Edinger
+Dziedzic
+Duerr
+Draughn
+Donoho
+Dimatteo
+Devos
+Dematteo
+Degnan
+Darlington
+Danis
+Dam
+Dahlstrom
+Dahlke
+Czajkowski
+Cumbie
+Culbert
+Crosier
+Croley
+Corry
+Clinger
+Cheshire
+Chalker
+Cephas
+Caywood
+Cavalier
+Capehart
+Cales
+Cadiz
+Bussiere
+Burriss
+Burkart
+Brundidge
+Bronstein
+Breeze
+Bradt
+Boydston
+Bostrom
+Borel
+Bolles
+Blay
+Blackwelder
+Bissett
+Bevers
+Bester
+Bernardino
+Benefiel
+Belote
+Beedle
+Beckles
+Baysinger
+Bassler
+Bartee
+Barlett
+Bargas
+Barefield
+Baptista
+Arterburn
+Armas
+Apperson
+Amoroso
+Amedee
+Zullo
+Zellner
+Yelton
+Willems
+Wilkin
+Wiggin
+Widman
+Welk
+Weingarten
+Walla
+Viers
+Vess
+Verdi
+Veazey
+Vannote
+Tullos
+Trudell
+Trower
+Trosper
+Trimm
+Trew
+Tousignant
+Topp
+Tocco
+Thoreson
+Terhune
+Tatom
+Suniga
+Sumter
+Steeves
+Stansell
+Soltis
+Sloss
+Slaven
+Sing
+Shisler
+Sheriff
+Shanley
+Servantes
+Selders
+Segrest
+Seese
+Seeber
+Schaible
+Savala
+Sartor
+Rutt
+Rumbaugh
+Ruis
+Roten
+Roessler
+Ritenour
+Riney
+Restivo
+Rene
+Renard
+Rakestraw
+Rake
+Rachal
+Quiros
+Pullin
+Prudhomme
+Primeaux
+Prestridge
+Presswood
+Ponte
+Polzin
+Poarch
+Pittenger
+Piggott
+Pickell
+Phaneuf
+Parvin
+Parmley
+Palmeri
+Paisley
+Ozment
+Ormond
+Ordaz
+Ono
+Olea
+Obanion
+Oakman
+Novick
+Nicklas
+Nemec
+Nappi
+Mund
+Morfin
+Mera
+Melgoza
+Melby
+Mcgoldrick
+Mcelwain
+Mcchristian
+Mccaw
+Marquart
+Marlatt
+Markovich
+Mahr
+Lupton
+Lucus
+Lorusso
+Lerman
+Leddy
+Leaman
+Leachman
+Lavalle
+Laduke
+Kummer
+Koury
+Konopka
+Koh
+Koepp
+Kloss
+Klock
+Khalil
+Kernan
+Kappel
+Jakes
+Inoue
+Hutsell
+Howle
+Honore
+Hole
+Hockman
+Hockaday
+Hiltz
+Hetherington
+Hesser
+Hershman
+Heng
+Heffron
+Headen
+Haskett
+Hartline
+Harned
+Guillemette
+Guglielmo
+Guercio
+Greenbaum
+Goris
+Glines
+Gilmour
+Gardella
+Gadd
+Gabler
+Gabbert
+Fuselier
+Freudenburg
+Fragoso
+Follis
+Flemings
+Feltman
+Febus
+Farren
+Fallis
+Evert
+Ekstrom
+Eastridge
+Dyck
+Dufault
+Dubreuil
+Dresser
+Drapeau
+Domingues
+Dolezal
+Dinkel
+Didonato
+Devitt
+Devane
+Demott
+Daughtrey
+Daubert
+Das
+Darrell
+Creason
+Crary
+Costilla
+Chipps
+Cheatwood
+Carmean
+Canton
+Caffrey
+Burgher
+Buker
+Brunk
+Brodbeck
+Brantner
+Brandy
+Bolivar
+Boerner
+Bodkin
+Biel
+Betty
+Bencomo
+Bellino
+Beliveau
+Beauvais
+Beaupre
+Baylis
+Baskett
+Barcus
+Barbera
+Baltz
+Asay
+Arney
+Arcuri
+Ankney
+Agostini
+Addy
+Zwilling
+Zubia
+Zollinger
+Zeitz
+Yard
+Yanes
+Winship
+Winningham
+Wickline
+Webre
+Waddington
+Vosburgh
+Vessels
+Verrett
+Vedder
+Varnum
+Vandeventer
+Vacca
+Usry
+Towry
+Touchet
+Tookes
+Tonkin
+Timko
+Tibbitts
+Thedford
+Tarleton
+Talty
+Talamantez
+Tafolla
+Sugg
+Strecker
+Stirling
+Steffan
+Spiva
+Slape
+Siemens
+Shatzer
+Seyler
+Seamans
+Schmaltz
+Schipper
+Sasso
+Sailor
+Ruppe
+Runner
+Royals
+Roudebush
+Ripple
+Riemer
+Richarson
+Revilla
+Reichenbach
+Ratley
+Railsback
+Quayle
+Poplin
+Poorman
+Ponton
+Polo
+Pollitt
+Poitras
+Piscitelli
+Piedra
+Pickles
+Pew
+Perera
+People
+Penwell
+Pelt
+Pauline
+Parkhill
+Paladino
+Ore
+Oram
+Olmo
+Oliveras
+Olivarria
+Ogorman
+Near
+Naron
+Na
+Muncie
+Mowbray
+Morones
+Moretti
+Monn
+Mitts
+Minks
+Minarik
+Mimms
+Milliron
+Millington
+Millhouse
+Messersmith
+Mcnett
+Mckinstry
+Mcgeorge
+Mcdill
+Mcateer
+Mazzeo
+Matchett
+Mahood
+Mabery
+Lundell
+Louden
+Losoya
+Lisk
+Lezama
+Leib
+Lebo
+Lanoue
+Lanford
+Lafortune
+Kump
+Krone
+Kreps
+Kott
+Kopecky
+Kolodziej
+Knuckles
+Kinman
+Kimmons
+Kelty
+Kaster
+Karlson
+Kania
+Jules
+Joyal
+Job
+Jenner
+Jasinski
+Jandreau
+Isenhour
+Hunziker
+Huhn
+Houde
+Houchins
+Holtman
+Hodo
+Heyman
+Hentges
+Hedberg
+Hayne
+Haycraft
+Harshbarger
+Harshaw
+Harriss
+Haring
+Hansell
+Hanford
+Handler
+Hamburg
+Hamblen
+Gunnell
+Groat
+Gorecki
+Gochenour
+Gleeson
+Genest
+Geiser
+Gabriele
+Fulghum
+Friese
+Fridley
+Freeborn
+Frailey
+Flaugher
+Fiala
+Ettinger
+Etheredge
+Espitia
+Eriksen
+Engelbrecht
+Engebretson
+Elie
+Eickhoff
+Edney
+Edelen
+Eberhard
+Eastin
+Eakes
+Driggs
+Doner
+Donaghy
+Disalvo
+Deshong
+Dahms
+Dahlquist
+Curren
+Cripe
+Cree
+Creager
+Corle
+Conatser
+Commons
+Coggin
+Coder
+Coaxum
+Closson
+Clodfelter
+Classen
+Chittenden
+Castilleja
+Casale
+Cartee
+Carriere
+Canup
+Canizales
+Burgoon
+Bunger
+Bugarin
+Buchanon
+Bruning
+Bruck
+Brookes
+Broadwell
+Brier
+Brekke
+Breese
+Bracero
+Bowley
+Bowersox
+Bose
+Bogar
+Blossom
+Blauser
+Blacker
+Bjorklund
+Belair
+Baumer
+Basler
+Barb
+Baltimore
+Baize
+Baden
+Auman
+Amundsen
+Amore
+Alvarenga
+Adan
+Adamczyk
+Yerkes
+Yerby
+Yawn
+Yamaguchi
+Worthey
+Wolk
+Wixom
+Wiersma
+Wieczorek
+Whiddon
+Weyer
+Wetherington
+Wein
+Watchman
+Warf
+Wansley
+Vesely
+Velazco
+Vannorman
+Valasquez
+Utz
+Urso
+Turco
+Turbeville
+Trivett
+Torrance
+Toothaker
+Toohey
+Tondreau
+Thaler
+Sylvain
+Swindler
+Swigert
+Swider
+Stiner
+Stever
+Steffes
+Stampley
+Stair
+Smidt
+Skeete
+Silvestre
+Shy
+Shutts
+Shock
+Shealey
+Seigler
+Schweizer
+Schuldt
+Schlichting
+Scherr
+Saulsberry
+Saner
+Rosin
+Rosato
+Roling
+Rohn
+Rix
+Rister
+Remley
+Remick
+Recinos
+Ramm
+Raabe
+Pursell
+Poythress
+Poli
+Pokorny
+Plum
+Pettry
+Petrey
+Petitt
+Penman
+Payson
+Paquet
+Pappalardo
+Outland
+Oscar
+Orenstein
+Nuttall
+Nuckols
+Nott
+Nimmo
+Murtagh
+Mousseau
+Moulder
+Mooneyhan
+Moak
+Minch
+Miera
+Mercuri
+Meighan
+Mcnelly
+Mcguffin
+Mccreery
+Mcclaskey
+Man
+Mainor
+Luongo
+Lundstrom
+Loughman
+Loose
+Lobo
+Lobb
+Linhart
+Liberty
+Lever
+Leu
+Leiter
+Lehoux
+Lehn
+Lares
+Lapan
+Langhorne
+Lamon
+Ladwig
+Ladson
+Kuzma
+Kreitzer
+Knop
+Keech
+Kea
+Kadlec
+Jo
+Jhonson
+Jantz
+Inglis
+Husk
+Hulme
+Housel
+Hofman
+Hillery
+Heidenreich
+Heaps
+Haslett
+Harting
+Hartig
+Hamler
+Halton
+Hallum
+Gutierres
+Guida
+Guerrier
+Grossi
+Gress
+Greenhalgh
+Gravelle
+Gow
+Goslin
+Gonyea
+Gipe
+Gerstner
+Gasser
+Garceau
+Gannaway
+Gama
+Gallop
+Gaiser
+Fullilove
+Foutz
+Fossum
+Flannagan
+Farrior
+Faller
+Ericksen
+Entrekin
+Enochs
+Englund
+Ellenberger
+Eastland
+Earwood
+Dudash
+Du
+Drozd
+Desoto
+Delph
+Dekker
+Dejohn
+Degarmo
+Defeo
+Defalco
+Deblois
+Dacus
+Cudd
+Crossen
+Crooms
+Cronan
+Costin
+Costanza
+Cordray
+Comerford
+Collie
+Colegrove
+Coldwell
+Claassen
+Chartrand
+Castiglione
+Carte
+Cardella
+Carberry
+Capp
+Capobianco
+Cangelosi
+Buch
+Brunell
+Brucker
+Brockett
+Brizendine
+Brinegar
+Brimer
+Brase
+Bosque
+Bonk
+Bolger
+Bohanon
+Bohan
+Blazek
+Berning
+Bergan
+Bennette
+Beauchemin
+Battiste
+Barra
+Balogh
+Avis
+Avallone
+Aubry
+Ashcroft
+Asencio
+Arledge
+Anchondo
+Amy
+Alvord
+Acheson
+Zaleski
+Yonker
+Wyss
+Wycoff
+Woodburn
+Wininger
+Winders
+Willmon
+Wiechmann
+Westley
+Weatherholt
+Warnick
+Wardle
+Warburton
+Volkert
+Virgin
+Villanveva
+Veit
+Vass
+Vanallen
+Tung
+Toribio
+Toothman
+Tiggs
+Thornsberry
+Thome
+Tepper
+Teeple
+Tebo
+Tassone
+Tann
+Sultan
+Stucker
+Stotler
+Stoneman
+Stehle
+Stanback
+Stallcup
+Spurr
+Speers
+Spada
+Solum
+Smolen
+Sinn
+Silvernail
+Sholes
+Shives
+Shain
+Secrest
+Seagle
+Schuette
+Schoch
+Schnieders
+Schild
+Schiavone
+Schiavo
+Scharff
+Santee
+Sandell
+Salvo
+Rollings
+Rollin
+Rivenburg
+Ritzman
+Rist
+Rio
+Ricardo
+Reynosa
+Retana
+Reiber
+Regnier
+Rarick
+Ransome
+Rall
+Propes
+Prall
+Poyner
+Ponds
+Poitra
+Plaster
+Pippins
+Pinion
+Piccolo
+Phu
+Perillo
+Penrose
+Pendergraft
+Pelchat
+Peed
+Patenaude
+Palko
+Odoms
+Oddo
+Novoa
+Noone
+Newburn
+Negri
+Nantz
+Mosser
+Moshier
+Molter
+Molinari
+Moler
+Millman
+Meurer
+Mendel
+Mcray
+Mcnicholas
+Mcnerney
+Mckillip
+Mcilvain
+Mcadory
+Matter
+Master
+Marmol
+Marinez
+Manzer
+Mankin
+Makris
+Majeski
+Magnus
+Maffei
+Luoma
+Luman
+Luebke
+Luby
+Lomonaco
+Loar
+Litchford
+Lintz
+Licht
+Levenson
+Legge
+Laughter
+Lanigan
+Krom
+Kreger
+Koop
+Kober
+Klima
+Kitterman
+Kinkead
+Kimbell
+Kilian
+Kibbe
+Kendig
+Kemmer
+Kash
+Jenkin
+Inniss
+Hurlbut
+Hunsucker
+Hugo
+Huckabee
+Hoxie
+Hoglund
+Hockensmith
+Hoadley
+Hinkel
+Higuera
+Herrman
+Heiner
+Hausmann
+Haubrich
+Hassen
+Hanlin
+Hallinan
+Haglund
+Hagberg
+Gullo
+Gullion
+Groner
+Greenwalt
+Grand
+Goodwill
+Gong
+Gobert
+Glowacki
+Glessner
+Gines
+Gildersleeve
+Gildea
+Gerke
+Gerhard
+Gebhard
+Gatton
+Gately
+Galasso
+Fralick
+Fouse
+Fluharty
+Faucette
+Fairfax
+Evanoff
+Elser
+Ellard
+Egerton
+Edie
+Ector
+Ebling
+Dunkel
+Duhart
+Drysdale
+Dostal
+Dorey
+Dolph
+Doles
+Dismukes
+Digregorio
+Digby
+Dewees
+Deramus
+Denniston
+Dennett
+Deloney
+Delaughter
+Darcy
+Cuneo
+Cumberland
+Crotts
+Crosswhite
+Cremeans
+Creasey
+Cottman
+Cothern
+Costales
+Cosner
+Corpus
+Cora
+Constable
+Colligan
+Cobble
+Clutter
+Chupp
+Chevez
+Chatmon
+Chaires
+Caplan
+Caffee
+Cabana
+Burrough
+Burditt
+Buckler
+Brunswick
+Brouillard
+Broady
+Bowlby
+Bouley
+Borgman
+Boltz
+Boddy
+Blackston
+Birdsell
+Bedgood
+Bate
+Basil
+Bartos
+Barriga
+Barrie
+Barna
+Barcenas
+Banach
+Baccus
+Auclair
+Ashman
+Arter
+Arendt
+Ansell
+Allums
+Allsop
+Allender
+Alber
+Albarran
+Adelson
+Zoll
+Wysong
+Wimbley
+Wildes
+Whitis
+Whitehill
+Whicker
+Weymouth
+Well
+Weldy
+Wark
+Wareham
+Waddy
+Viveiros
+Vito
+Vides
+Vecchio
+Vath
+Vandoren
+Vanderhoof
+Unrein
+Uecker
+Tsan
+Trepanier
+Tregre
+Torkelson
+Ton
+Tobler
+Tineo
+Timmer
+Swopes
+Swofford
+Sweeten
+Swarts
+Summerfield
+Sumler
+Stucky
+Strozier
+Stigall
+Stickel
+Stennis
+Stelzer
+Steely
+Solar
+Slayden
+Skillern
+Shurtz
+Shelor
+Shellenbarger
+Shand
+Shabazz
+Seo
+Scroggs
+Schwandt
+Schrecengost
+Schoenrock
+Schirmer
+Sandridge
+Ruzicka
+Rozek
+Rowlands
+Roser
+Rosendahl
+Romanowski
+Romaine
+Rolston
+Rink
+Riggio
+Reichman
+Redondo
+Reay
+Rawlinson
+Raskin
+Raine
+Quandt
+Purpura
+Purdue
+Pruneda
+Prevatte
+Prettyman
+Pinedo
+Pierro
+Pidgeon
+Phillippi
+Pfeil
+Penix
+Peasley
+Paro
+Overall
+Ospina
+Ortegon
+Ogata
+Ogara
+Normandin
+Nordman
+Nims
+Nassar
+Motz
+Morlan
+Mooring
+Moles
+Moir
+Mizrahi
+Mire
+Minaya
+Millwood
+Mikula
+Messmer
+Meikle
+Mctaggart
+Mcgonagle
+Mcewan
+Mccasland
+Mccane
+Mccaffery
+Mcalexander
+Mattocks
+Mattie
+Matranga
+Martone
+Markland
+Maravilla
+Manno
+Manly
+Mancha
+Mallery
+Magno
+Lorentz
+Locklin
+Livingstone
+Lipford
+Lininger
+Line
+Liao
+Lepley
+Leming
+Lemelin
+Leadbetter
+Lawhon
+Lattin
+Langworthy
+Lampman
+Lambeth
+Lamarr
+Lahey
+Krajewski
+Klopp
+Kinnison
+Kestner
+Kerry
+Kennell
+Karim
+Jozwiak
+Jakubowski
+Jagger
+Ivery
+Ishmael
+Iliff
+Iddings
+Hudkins
+Houseman
+Holz
+Holderman
+Hoehne
+Highfill
+Hiett
+Heskett
+Heldt
+Hedman
+Hayslett
+Hatchell
+Hasse
+Hamon
+Hamada
+Hakala
+Haislip
+Haffey
+Hackbarth
+Guo
+Gullickson
+Guerrette
+Guan
+Greenblatt
+Goudreau
+Gongora
+Godbout
+Glaude
+Gills
+Gillison
+Gigliotti
+Gargano
+Gallucci
+Galli
+Galante
+Frasure
+Fodor
+Fizer
+Fishburn
+Finkbeiner
+Finck
+Fager
+Estey
+Espiritu
+Eppinger
+Epperly
+Emig
+Eckley
+Dray
+Dorsch
+Dille
+Devita
+Deslauriers
+Demery
+Delorme
+Delbosque
+Dauphin
+Dantonio
+Curd
+Crume
+Crown
+Cozad
+Cossette
+Comacho
+Climer
+Chadbourne
+Cespedes
+Cayton
+Castaldo
+Carpino
+Carls
+Capozzi
+Canela
+Cadet
+Buzard
+Busick
+Burlison
+Brinkmann
+Bridgeforth
+Bourbeau
+Bornstein
+Boots
+Bonfiglio
+Boice
+Boese
+Biondi
+Bilski
+Betton
+Berwick
+Berlanga
+Behan
+Becraft
+Barrientez
+Banh
+Balke
+Balderrama
+Bahe
+Bachand
+Atlas
+Armer
+Arceo
+Aliff
+Alatorre
+Zermeno
+Zane
+Younce
+You
+Yeoman
+Yamasaki
+Wroten
+Worm
+Woodby
+Winer
+Wilmer
+Willits
+Wilcoxon
+Wehmeyer
+Waterbury
+Wass
+Wann
+Wake
+Wachtel
+Vizcarra
+Vince
+Victory
+Veitch
+Vanderbilt
+Vallone
+Vallery
+Ureno
+Tyer
+Tipps
+Tiedeman
+Theberge
+Texeira
+Taub
+Tapscott
+Stutts
+Stults
+Stukes
+Staff
+Spink
+Sottile
+Smithwick
+Slane
+Simeone
+Silvester
+Siegrist
+Shiffer
+Sheedy
+Sheaffer
+Severin
+Sellman
+Scotto
+Schupp
+Schueller
+Schreier
+Schoolcraft
+Schoenberger
+Schnabel
+Sangster
+Samford
+Saliba
+Ryles
+Ryans
+Rossetti
+Rodriguz
+Risch
+Riel
+Rezendes
+Rester
+Rencher
+Recker
+Rathjen
+Profitt
+Poteete
+Polizzi
+Perrigo
+Patridge
+Osby
+Orvis
+Opperman
+Oppenheim
+Onorato
+Olaughlin
+Ohagan
+Ogles
+Oehler
+Obyrne
+Nuzzo
+Nickle
+Nease
+Neagle
+Navarette
+Nagata
+Musto
+Morning
+Morison
+Montz
+Mogensen
+Mizer
+Miraglia
+Mingus
+Migliore
+Merideth
+Menges
+Mellor
+Mcnear
+Mcnab
+Mcloud
+Mcelligott
+Mccollom
+Maynes
+Marquette
+Markowski
+Marcantonio
+Mar
+Maldanado
+Makin
+Macey
+Lundeen
+Lovin
+Longino
+Lisle
+Linthicum
+Limones
+Lesure
+Lesage
+Leisure
+Lauver
+Laubach
+Latshaw
+Lary
+Lapham
+Lacoste
+Lacher
+Kutcher
+Knickerbocker
+Klos
+Klingler
+Kleiman
+Kittleson
+Kimbrel
+Kimberly
+Kemmerer
+Kelson
+Keese
+Kam
+Kallas
+Jurgensen
+Junkins
+Juneau
+Juergens
+Jolliff
+Jelks
+Janicki
+Jang
+Innocent
+Ingles
+Inge
+Huguley
+Huggard
+Howton
+Hone
+Holford
+Holding
+Hogle
+Hipple
+Heimbach
+Heider
+Heidel
+Havener
+Hattaway
+Harrah
+Hanscom
+Hankinson
+Hamdan
+Gridley
+Goulette
+Goulart
+Goodspeed
+Goodrow
+Go
+Girardi
+Gent
+Gautreau
+Ganz
+Gandara
+Gamblin
+Galipeau
+Fyffe
+Furrow
+Fulp
+Fricks
+Frase
+Frandsen
+Fout
+Foulks
+Fouche
+Foskey
+Forgey
+Foor
+Fobbs
+Finklea
+Fincham
+Figueiredo
+Festa
+Ferrier
+Fellman
+Eslick
+Eilerman
+Eckart
+Eaglin
+Dunfee
+Dumond
+Drewry
+Douse
+Domino
+Dimick
+Diener
+Dickert
+Deines
+Degree
+Declue
+Daw
+Dattilo
+Danko
+Custodio
+Cuccia
+Crunk
+Crispin
+Corp
+Cornwall
+Corea
+Coppin
+Considine
+Coniglio
+Conboy
+Collar
+Cockrum
+Clute
+Clewis
+Claude
+Christiano
+Channell
+Channel
+Cerrato
+Cecere
+Catoe
+Castillon
+Castile
+Carstarphen
+Carmouche
+Caperton
+Buteau
+Bury
+Bumpers
+Brey
+Brenton
+Brazeal
+Brassard
+Brass
+Braga
+Bradham
+Bourget
+Borrelli
+Borba
+Boothby
+Bohr
+Bohm
+Boehme
+Bodin
+Bloss
+Blocher
+Bizzell
+Bieker
+Berthelot
+Bernardini
+Berends
+Benard
+Belser
+Baze
+Bartling
+Barrientes
+Barras
+Barcia
+Banfield
+Aurand
+Artman
+Arnott
+Arend
+Ardis
+Amon
+Almaguer
+Allee
+Albarado
+Alameda
+Abdo
+Zuehlke
+Zoeller
+Yokoyama
+Yocom
+Wyllie
+Woolum
+Wint
+Winland
+Wink
+Wilner
+Wilmes
+Whitlatch
+Westervelt
+Walthall
+Walkowiak
+Walburn
+Viviano
+Vanderhoff
+Valez
+Ugalde
+Trumbull
+Todaro
+Tilford
+Tidd
+Tibbits
+Terranova
+Templeman
+Tannenbaum
+Talmage
+Tabarez
+Swearengin
+Swartwood
+Svendsen
+Strum
+Strack
+Storie
+Stockard
+Steinbeck
+Starns
+Stanko
+Stankiewicz
+Stacks
+Stach
+Sproles
+Spenser
+Smotherman
+Slusser
+Sinha
+Silber
+Siefert
+Siddiqui
+Shuff
+Sherburne
+Seldon
+Seddon
+Schweigert
+Schroeter
+Schmucker
+Saffold
+Rutz
+Rundle
+Rosinski
+Rosenow
+Rogalski
+Ridout
+Rhymer
+Replogle
+Regina
+Reda
+Raygoza
+Ratner
+Rascoe
+Rahm
+Quincy
+Quast
+Pry
+Pressnell
+Predmore
+Pou
+Porto
+Pleasants
+Pigford
+Pavone
+Patnaude
+Parramore
+Papadopoulos
+Palmatier
+Ouzts
+Oshields
+Ortis
+Olmeda
+Olden
+Okamoto
+Norby
+Nitz
+Niebuhr
+Nevius
+Neiman
+Neidig
+Neece
+Murawski
+Mroz
+Moylan
+Moultry
+Mosteller
+Moring
+Morganti
+Mook
+Moffet
+Mettler
+Merlo
+Mengel
+Mendelsohn
+Meli
+Melchior
+Mcmeans
+Mcfaddin
+Mccullers
+Mccollister
+Mccloy
+Mcclaine
+Maury
+Maser
+Martelli
+Manthey
+Malkin
+Maio
+Magwood
+Maginnis
+Mabon
+Luton
+Lusher
+Lucht
+Lobato
+Levis
+Letellier
+Legendre
+Laurel
+Latson
+Larmon
+Largo
+Landreneau
+Landgraf
+Lamberson
+Kurland
+Kresge
+Korman
+Korando
+Klapper
+Kitson
+Kinyon
+Kincheloe
+Kawamoto
+Kawakami
+Jenney
+Jeanpierre
+Ivers
+Issa
+Ince
+Hugh
+Hug
+Honda
+Hollier
+Hollars
+Hoerner
+Hodgkinson
+Hiott
+Hibbitts
+Herlihy
+Henricks
+Heavner
+Hayhurst
+Harvill
+Harewood
+Hanselman
+Hanning
+Gwyn
+Gustavson
+Grounds
+Grizzard
+Grinder
+Graybeal
+Gravley
+Gorney
+Goll
+Goehring
+Godines
+Gobeil
+Glickman
+Giuliano
+Gimbel
+Gift
+Geib
+Gayhart
+Gatti
+Gains
+Gadberry
+Frei
+Fraise
+Fouch
+Forst
+Forsman
+Folden
+Fogleman
+Figaro
+Fetty
+Feely
+Fabry
+Eury
+Estill
+Epling
+Elamin
+Echavarria
+Dutil
+Duryea
+Dumais
+Drago
+Downard
+Douthit
+Doolin
+Dobos
+Dison
+Dinges
+Diebold
+Desilets
+Deshazo
+Depaz
+Degennaro
+Dall
+Cyphers
+Cryer
+Croce
+Crisman
+Credle
+Coriell
+Copp
+Coop
+Compos
+Colmenero
+Cogar
+Cliff
+Chapel
+Carnevale
+Campanella
+Caley
+Calderone
+Burtch
+Brouwer
+Brehmer
+Brassell
+Brafford
+Bourquin
+Bourn
+Bohnert
+Blewett
+Blass
+Blakes
+Bhakta
+Besser
+Berge
+Bellis
+Balfour
+Avera
+Austria
+Applin
+Ammon
+Alsop
+Aleshire
+Akbar
+Zoller
+Zapien
+Wymore
+Wyble
+Wolken
+Wix
+Wickstrom
+Whobrey
+Whigham
+Westerlund
+Welsch
+Weisser
+Weisner
+Weinstock
+Wehner
+Watlington
+Wakeland
+Wafer
+Virgen
+Victorino
+Veltri
+Veith
+Urich
+Uresti
+Umberger
+Twedt
+Tuohy
+Tschida
+Trumble
+Troia
+Tristan
+Trimmer
+Topps
+Tonn
+Tiernan
+Threet
+Thrall
+Thetford
+Teneyck
+Tartaglia
+Swords
+Strohl
+Streater
+Strausbaugh
+Stradley
+Stonecipher
+Steadham
+Stansel
+Stalcup
+Stabile
+Sprenger
+Spradley
+Speier
+Southwood
+Sorrels
+Slezak
+Skow
+Sirmans
+Simental
+Silk
+Sifford
+Sievert
+Shover
+Sheley
+Selzer
+Scriven
+Schwindt
+Schwan
+Schroth
+Saylors
+Saragosa
+Sant
+Salaam
+Saephan
+Routt
+Rousey
+Ros
+Rolfes
+Rieke
+Rieder
+Richeson
+Redinger
+Rasnick
+Rapoza
+Rambert
+Rafael
+Quist
+Pyron
+Punch
+Pullman
+Przybylski
+Pridmore
+Pooley
+Pines
+Perkinson
+Perine
+Perham
+Pecor
+Peavler
+Partington
+Panton
+Oliverio
+Olague
+Ohman
+Ohearn
+Noyola
+Nicolai
+Nebel
+Murtha
+Muff
+Mowrey
+Moroney
+Morgenstern
+Morant
+Monty
+Monsour
+Mohammad
+Moffit
+Mijares
+Meriwether
+Mendieta
+Melendrez
+Mejorado
+Mckittrick
+Mckey
+Mckenny
+Mckelvy
+Mckechnie
+Mcelvain
+Mccoin
+Mazzarella
+Mazon
+Maurin
+Matthies
+Maston
+Maske
+Marzano
+Marmon
+Marburger
+Mangus
+Mangino
+Mallet
+Luo
+Losada
+Londono
+Lobdell
+Lipson
+Lesniak
+Leighty
+Lei
+League
+Lavallie
+Lareau
+Laperle
+Lape
+Laforce
+Laffey
+Kuehner
+Kravitz
+Kowalsky
+Kohr
+Kinsman
+Keppler
+Kennemer
+Keiper
+Keely
+Kaler
+Jun
+Jelinek
+Jarnagin
+Issac
+Isakson
+Hypes
+Hutzler
+Huls
+Horak
+Hitz
+Hice
+Herrell
+Henslee
+Heitz
+Heiss
+Heiman
+Hasting
+Hartwick
+Harmer
+Harland
+Hammontree
+Haldeman
+Hakes
+Guse
+Guillotte
+Guard
+Groleau
+Greve
+Greenough
+Golub
+Golson
+Goldschmidt
+Golder
+Godbolt
+Gilmartin
+Gies
+Gibby
+Geren
+Genthner
+Gendreau
+Gemmill
+Gaymon
+Galyean
+Galeano
+Friar
+Folkerts
+Fleeman
+Fitzgibbons
+Ferranti
+Felan
+Farrand
+Eoff
+Enger
+Engels
+Ducksworth
+Duby
+Dry
+Drumheller
+Douthitt
+Doris
+Donis
+Dixion
+Dittrich
+Dials
+Dessert
+Descoteaux
+Depaul
+Denker
+Demuth
+Demelo
+Delacerda
+Deforge
+Danos
+Dalley
+Daigneault
+Cybulski
+Crystal
+Cristobal
+Cothren
+Corns
+Corkery
+Copas
+Coco
+Clubb
+Clore
+Chitty
+Chichester
+Chery
+Charon
+Chamber
+Chace
+Catanzaro
+Castonguay
+Cassella
+Caroll
+Carlberg
+Cammarata
+Calle
+Cajigas
+Byas
+Buzbee
+Busey
+Burling
+Bufkin
+Brzezinski
+Brun
+Brickner
+Brabham
+Boller
+Bodily
+Bockman
+Bleich
+Blakeman
+Bisbee
+Bier
+Bezanson
+Bevilacqua
+Besaw
+Berrian
+Berkeley
+Bequette
+Beauford
+Baumgarten
+Baudoin
+Batie
+Basaldua
+Bardin
+Bangert
+Banes
+Backlund
+Avitia
+Artz
+Archey
+Apel
+Amico
+Alam
+Aden
+Zebrowski
+Yokota
+Wormley
+Wootton
+Woodie
+Womac
+Wiltz
+Wigington
+Whitehorn
+Whisman
+Weisgerber
+Weigle
+Weedman
+Watkin
+Wasilewski
+Wadlington
+Wadkins
+Viverette
+Vidaurri
+Vidales
+Vezina
+Vanleer
+Vanhoy
+Vanguilder
+Vanbrunt
+Uy
+Updegraff
+Tylor
+Trinkle
+Touchette
+Tilson
+Tilman
+Tengan
+Tarkington
+Surrett
+Super
+Summy
+Streetman
+Straughter
+Steere
+Stalling
+Spruell
+Spadaro
+Solley
+Smathers
+Silvera
+Siems
+Shreffler
+Sholar
+Selden
+Schaper
+Samayoa
+Ruggeri
+Rowen
+Rosso
+Rosenbalm
+Roosevelt
+Roose
+Ronquillo
+Rogowski
+Rexford
+Repass
+Renzi
+Renick
+Renda
+Rehberg
+Reaper
+Ranck
+Raffa
+Rackers
+Raap
+Pugsley
+Puglisi
+Prinz
+Primus
+Pounders
+Pon
+Pompa
+Plasencia
+Pipkins
+Pillar
+Petrosky
+Pelley
+Pauls
+Pauli
+Parkison
+Parisien
+Pangle
+Pancoast
+Palazzolo
+Owenby
+Overbay
+Orris
+Orlowski
+Nipp
+Newbern
+Nedd
+Nealon
+Najar
+Mysliwiec
+Myron
+Myres
+Musson
+Murrieta
+Munsell
+Mumma
+Muldowney
+Moyle
+Mowen
+Mose
+Morejon
+Moodie
+Monier
+Mikkelsen
+Miers
+Metzinger
+Melin
+Mcquay
+Mcpeek
+Mcneeley
+Mcglothin
+Mcghie
+Mcdonell
+Mccumber
+Mccranie
+Mcbean
+Mayhugh
+Marts
+Marenco
+Manges
+Lynam
+Lupien
+Luff
+Luebbert
+Loh
+Loflin
+Lococo
+Loch
+Lis
+Linke
+Lightle
+Lewellyn
+Leishman
+Lebow
+Lebouef
+Leanos
+Lanz
+Landy
+Landaverde
+Lacefield
+Kyler
+Kuebler
+Kropf
+Kroeker
+Kluesner
+Klass
+Kimberling
+Kilkenny
+Kiker
+Ketter
+Kelemen
+Keasler
+Kawamura
+Karst
+Kardos
+Jeremiah
+Jared
+Igo
+Huseman
+Huseby
+Hurlbert
+Huard
+Hottinger
+Hornberger
+Hopps
+Holdsworth
+Hensen
+Heilig
+Heeter
+Harpole
+Haak
+Gutowski
+Gunnels
+Grimmer
+Grieve
+Gravatt
+Granderson
+Gotcher
+Gleaves
+Genao
+Garfinkel
+Frerichs
+Foushee
+Flanery
+Finnie
+Feldt
+Fagin
+Ewalt
+Ellefson
+Eiler
+Eckhart
+Eastep
+Dwight
+Digirolamo
+Didomenico
+Devera
+Delavega
+Defilippo
+Debusk
+Daub
+Damiani
+Cupples
+Cuddy
+Crofoot
+Courter
+Coto
+Costigan
+Corning
+Corman
+Corlett
+Cooperman
+Collison
+Coghlan
+Cobbins
+Coady
+Coachman
+Clothier
+Client
+Clear
+Cipolla
+Chmielewski
+Chiodo
+Chatterton
+Chappelle
+Chairez
+Ceron
+Casperson
+Casler
+Casados
+Carrow
+Carolina
+Carlino
+Carico
+Cardillo
+Caouette
+Canto
+Canavan
+Cambra
+Byard
+Buterbaugh
+Buse
+Bucy
+Buckwalter
+Bubb
+Bryd
+Brissette
+Brault
+Bradwell
+Boshears
+Borchert
+Blansett
+Blanch
+Blade
+Biondo
+Bilbo
+Biehl
+Bessey
+Berta
+Belles
+Bella
+Beeks
+Beekman
+Beaufort
+Bayliss
+Bardsley
+Avilla
+Astudillo
+Ardito
+Anwar
+Antunez
+Amen
+Aderholt
+Abate
+Yowell
+Yin
+Yearby
+Ye
+Wurst
+Woolverton
+Woolbright
+Wildermuth
+Whittenburg
+Whitely
+Wetter
+Wetherbee
+Wenz
+Welliver
+Welling
+Welcome
+Wason
+Warrior
+Warlick
+Voorhies
+Vivier
+Villines
+Vida
+Verde
+Veiga
+Varghese
+Vanwyk
+Vanwingerden
+Vanhorne
+Umstead
+Twiggs
+Tusing
+Trego
+Tompson
+Tinkle
+Thoman
+Thole
+Tatman
+Tartt
+Suda
+Studley
+Strock
+Strawbridge
+Stokely
+Stec
+Stang
+Stalter
+Speidel
+Spafford
+Spade
+Sontag
+Sokolowski
+Skillman
+Skelley
+Skalski
+Sison
+Sippel
+Sinquefield
+Sin
+Siegle
+Sher
+Sharrow
+Setliff
+Sera
+Sellner
+Selig
+Seibold
+Seery
+Scriber
+Schull
+Schrupp
+Schippers
+Say
+Saulsbury
+Sao
+Santillo
+Sanor
+Sancho
+Rufus
+Rubalcaba
+Roosa
+Ronk
+Robbs
+Roache
+River
+Riebe
+Reinoso
+Quin
+Prude
+Preuss
+Pottorff
+Pontiff
+Plouffe
+Picou
+Picklesimer
+Pettyjohn
+Petti
+Penaloza
+Parmelee
+Pardee
+Palazzo
+Overholt
+Ogawa
+Ofarrell
+Nova
+Nolting
+Noda
+Nicola
+Nickson
+Nevitt
+Neveu
+Navarre
+Nam
+Murrow
+Munz
+Mulloy
+Monzo
+Milliman
+Metivier
+Merlino
+Mcpeters
+Mckissack
+Mckeen
+Mcgurk
+Mcfee
+Mcfarren
+Mcelwee
+Mceachin
+Mcdonagh
+Mccarville
+Mayhall
+Mattoon
+Martello
+Marconi
+Marbury
+Mao
+Manzella
+Maly
+Malec
+Maitland
+Maheu
+Maclennan
+Lyke
+Luera
+Loyola
+Lowenstein
+Losh
+Lopiccolo
+Longacre
+Loman
+Loden
+Loaiza
+Lieber
+Libbey
+Lenhardt
+Lefebre
+Lauterbach
+Lauritsen
+Lass
+Larocco
+Larimer
+Lansford
+Lanclos
+Lamay
+Lal
+Kulikowski
+Kriebel
+Kosinski
+Kleinman
+Kleiner
+Kleckner
+Kistner
+Kissner
+Kissell
+Kilroy
+Kenna
+Keisler
+Keeble
+Keaney
+Kale
+Joly
+Jimison
+Jeans
+Ikner
+Hursey
+Hruska
+Hove
+Hou
+Host
+Hosking
+Hoose
+Holle
+Hoeppner
+Hittle
+Hitchens
+Hirth
+Hinerman
+Hilario
+Higby
+Hertzog
+Hentz
+Hensler
+Heist
+Heier
+Hegg
+Hassel
+Harpe
+Hara
+Hank
+Hain
+Hagopian
+Grimshaw
+Grado
+Gowin
+Gowans
+Googe
+Goodlow
+Goering
+Gleaton
+Gidley
+Giannone
+Gascon
+Garneau
+Gambrel
+Galaz
+Fuentez
+Frisina
+Fresquez
+Fraher
+Fitting
+Feuerstein
+Felten
+Everman
+Estell
+Ertel
+Erazo
+Ensign
+Endo
+Ellerman
+Eichorn
+Edgell
+Ebron
+Eaker
+Dundas
+Duncanson
+Duchene
+Ducan
+Dombroski
+Doman
+Dock
+Dickison
+Dewoody
+Deloera
+Delahoussaye
+Dejean
+Degroat
+Decaro
+Dearmond
+Dashner
+Dales
+Crossett
+Cressey
+Cowger
+Courts
+Court
+Cornette
+Corbo
+Coplin
+Coover
+Condie
+Cokley
+Cicero
+Ceaser
+Cannaday
+Callanan
+Cadle
+Buscher
+Bullion
+Bucklin
+Bruening
+Bruckner
+Brose
+Branan
+Bradway
+Botsford
+Bortz
+Borelli
+Bonetti
+Bolan
+Boerger
+Bloomberg
+Bingman
+Bilger
+Berns
+Beringer
+Beres
+Beets
+Beede
+Beaudet
+Beachum
+Baughn
+Bator
+Bastien
+Basquez
+Barreiro
+Barga
+Baratta
+Balser
+Baillie
+Axford
+Attebery
+Arakaki
+Annunziata
+Andrzejewski
+Ament
+Amendola
+Adcox
+Abril
+Zenon
+Zeitler
+Zang
+Zambrana
+Ybanez
+Yagi
+Wolak
+Wilcoxson
+Whitesel
+Whitehair
+Weyand
+Westendorf
+Welke
+Weinmann
+Wei
+Weesner
+Weekes
+Wedel
+Wedding
+Weatherall
+Warthen
+Vose
+Villalta
+Vila
+Viator
+Vaz
+Valtierra
+Urbanek
+Tulley
+Trojanowski
+Trapani
+Toups
+Torpey
+Tomita
+Tindal
+Tieman
+Tevis
+Tedrow
+Taul
+Tash
+Tammaro
+Sylva
+Swiderski
+Sweeting
+Sund
+Stutler
+Stocking
+Stich
+Sterns
+Stegner
+Stalder
+Splawn
+Speirs
+Southwell
+Soltys
+Smead
+Slye
+Skipworth
+Sipos
+Simmerman
+Sigmund
+Sidhu
+Shuffler
+Shingleton
+Shadwick
+Sermons
+Seefeldt
+Scipio
+Schwanke
+Schreffler
+Schiro
+Scheiber
+Sandoz
+Samsel
+Ruddell
+Royse
+Rouillard
+Rotella
+Rosalez
+Romriell
+Rommel
+Rizer
+Riner
+Rickards
+Rhoton
+Rhem
+Reppert
+Rayl
+Raulston
+Raposo
+Rapier
+Rainville
+Radel
+Quinney
+Purdie
+Puffer
+Pizzo
+Pincus
+Petrus
+Pendelton
+Pendarvis
+Peltz
+Peguero
+Peete
+Patricio
+Patchett
+Parrino
+Papke
+Pam
+Palafox
+Ottley
+Ostby
+Oritz
+Oren
+Ogan
+Odegaard
+Oatman
+Noell
+Nida
+Nicoll
+Newhall
+Newbill
+Netzer
+Nettleton
+Neblett
+Murley
+Mungo
+Mulhall
+Mosca
+Morissette
+Morford
+Montag
+Monsen
+Mitzel
+Miskell
+Minder
+Mehaffey
+Mcquillen
+Mclennan
+Mcgrail
+Mccreight
+Mayville
+Maysonet
+Maust
+Mathieson
+Mastrangelo
+Maskell
+Martina
+Manz
+Malmberg
+Makela
+Madruga
+Luz
+Lotts
+Longnecker
+Logston
+Littell
+Liska
+Lindauer
+Lillibridge
+Levron
+Letchworth
+Lesh
+Leffel
+Leday
+Leamon
+Laura
+Kulas
+Kula
+Kucharski
+Kromer
+Kraatz
+Konieczny
+Konen
+Komar
+Kivett
+Kirts
+Kinnear
+Kersh
+Keithley
+Keifer
+Judah
+Jimenes
+Jeppesen
+Jasmin
+Jansson
+Huntsberry
+Hund
+Huitt
+Huffine
+Hosford
+Hopes
+Holmstrom
+Hollen
+Hodgin
+Hirschman
+Hiltner
+Hilliker
+Hibner
+Hennis
+Helt
+Heidelberg
+Heger
+Heer
+Hartness
+Hardrick
+Halladay
+Gula
+Guillaume
+Guerriero
+Grunewald
+Grosse
+Griffeth
+Grenz
+Grassi
+Grandison
+Ginther
+Gimenez
+Gillingham
+Gillham
+Gess
+Gelman
+Gearheart
+Gaskell
+Gariepy
+Gamino
+Gallien
+Galentine
+Fuquay
+Froman
+Froelich
+Friedel
+Foos
+Fomby
+Focht
+Flythe
+Fiqueroa
+Filson
+Filip
+Fierros
+Fett
+Fedele
+Fasching
+Farney
+Fargo
+Everts
+Even
+Etzel
+Elzey
+Eichner
+Eger
+Eatman
+Ducker
+Duchesne
+Donati
+Domenech
+Dollard
+Dodrill
+Dinapoli
+Denn
+Delfino
+Delcid
+Delaune
+Delatte
+Deems
+Daluz
+Cusson
+Cullison
+Cue
+Cuadrado
+Crumrine
+Cruickshank
+Crosland
+Croll
+Criddle
+Crepeau
+Coutu
+Couey
+Cort
+Coppinger
+Collman
+Cockburn
+Coca
+Clayborne
+Claflin
+Cissell
+Chowdhury
+Chicoine
+Chenier
+Causby
+Caulder
+Cassano
+Casner
+Cardiel
+Burner
+Brunton
+Bruch
+Broxton
+Brosius
+Brooking
+Branco
+Bracco
+Bourgault
+Bosserman
+Books
+Bonet
+Bolds
+Bolander
+Bohman
+Boelter
+Blohm
+Blea
+Blaise
+Bischof
+Billie
+Beus
+Bellew
+Bastarache
+Bast
+Bartolome
+Bark
+Barcomb
+Barco
+Balls
+Balk
+Balas
+Bakos
+Avey
+Atnip
+Ashbrook
+Arno
+Arbour
+Aquirre
+Appell
+Aldaco
+Alcazar
+Alban
+Ahlstrom
+Abadie
+Zylstra
+Zick
+Zheng
+Yother
+Wyse
+Wunsch
+Whitty
+Weist
+Vrooman
+Vine
+Villalon
+Vidrio
+Vavra
+Vasbinder
+Vanmatre
+Vandorn
+Ugarte
+Turberville
+Tuel
+Trogdon
+Town
+Toupin
+Toone
+Tolleson
+Tinkham
+Tinch
+Tiano
+Teston
+Teer
+Tea
+Tawney
+Taplin
+Tant
+Tansey
+Swayne
+Sutcliffe
+Sunderman
+Suits
+Strothers
+Stromain
+Stork
+Stoneburner
+Stolte
+Stolp
+Stoehr
+Stingley
+Stegman
+Stangl
+Spinella
+Spier
+Soules
+Sommerfield
+Sipp
+Simek
+Siders
+Shufelt
+Shue
+Shor
+Shires
+Shellenberger
+Sheely
+Service
+Sepe
+Seaberg
+Schwing
+Scherrer
+Scalzo
+Saver
+Sasse
+Sarvis
+Santora
+Sansbury
+Salls
+Saleem
+Ryland
+Rybicki
+Ruggieri
+Rothenberg
+Rosenstein
+Roquemore
+Rollison
+Rodden
+Rivet
+Rita
+Ridlon
+Riche
+Riccardi
+Reiley
+Regner
+Rech
+Rayo
+Rawley
+Ranger
+Raff
+Radabaugh
+Quon
+Quill
+Privette
+Prange
+Pickrell
+Perino
+Penning
+Pankratz
+Orlandi
+Nyquist
+Norrell
+Noren
+Naples
+Nale
+Nakashima
+Musselwhite
+Murrin
+Murch
+Mullinix
+Mullican
+Mullan
+Morneau
+Mondor
+Molinar
+Mo
+Minjares
+Minix
+Mingle
+Minchew
+Mill
+Milewski
+Mikkelson
+Mifflin
+Messing
+Merkley
+Meis
+Meas
+Mcroy
+Mcphearson
+Mcneel
+Mcmunn
+Mcmorrow
+Mcdorman
+Mccroskey
+Mccoll
+Mcclusky
+Mcclaran
+Mccampbell
+Mazzariello
+Mauzy
+Mauch
+Mastro
+Martinek
+Marsala
+Marcantel
+Mahle
+Lyda
+Lucius
+Luciani
+Lubbers
+Louder
+Lobel
+Linsey
+Linch
+Liller
+Legros
+Layden
+Lapine
+Lansberry
+Lage
+Laforest
+Labriola
+Koga
+Knupp
+Klimek
+Kittinger
+Kirchoff
+Kinzel
+Killinger
+Kilbourne
+Ketner
+Kepley
+Kemble
+Kells
+Kear
+Kaya
+Karsten
+Kaneshiro
+Kamm
+Joines
+Joachim
+Janelle
+Jacobus
+Iler
+Holgate
+Hoar
+Hisey
+Hird
+Hilyard
+Heslin
+Herzberg
+Hennigan
+Hegland
+Hartl
+Haner
+Handel
+Gualtieri
+Greenly
+Grasser
+Gran
+Goetsch
+Godbold
+Gilland
+Gidney
+Gibney
+Giancola
+Gettinger
+Garzon
+Garret
+Galle
+Galgano
+Gaier
+Gaertner
+Fuston
+Freel
+Fortes
+Flock
+Fiorillo
+Figgs
+Fenstermacher
+Fedler
+Facer
+Fabiano
+Evins
+Eusebio
+Euler
+Esquer
+Enyeart
+Elem
+Eisenhower
+Eich
+Edgerly
+Durocher
+Durgan
+Duffin
+Drolet
+Drewes
+Dotts
+Dossantos
+Dolly
+Dockins
+Dirksen
+Difiore
+Dierks
+Dickerman
+Dice
+Dery
+Denault
+Demaree
+Delmonte
+Delcambre
+Days
+Daulton
+Darst
+Dahle
+Curnutt
+Cully
+Culligan
+Cueva
+Crosslin
+Croskey
+Cromartie
+Crofts
+Covin
+Coutee
+Countess
+Cost
+Coppa
+Coogan
+Condrey
+Concannon
+Coger
+Cloer
+Clatterbuck
+Cieslak
+Chumbley
+Choudhury
+Chiaramonte
+Charboneau
+Chai
+Carneal
+Cappello
+Campisi
+Callicoat
+Burgoyne
+Bucholz
+Brumback
+Brosnan
+Brogden
+Broder
+Brendle
+Breece
+Bown
+Bou
+Boser
+Bondy
+Bolster
+Boll
+Bluford
+Blandon
+Biscoe
+Bevill
+Bence
+Battin
+Basel
+Bartram
+Barnaby
+Barmore
+Balbuena
+Badgley
+Backstrom
+Auyeung
+Ater
+Arrellano
+Arant
+Ansari
+Alling
+Alejandre
+Alcock
+Alaimo
+Aguinaldo
+Aarons
+Zurita
+Zeiger
+Zawacki
+Yutzy
+Yarger
+Wygant
+Wurm
+Wuest
+Wolfram
+Witherell
+Wisneski
+Whitby
+Whelchel
+Weisz
+Weisinger
+Weishaar
+Wehr
+Wedge
+Waxman
+Waldschmidt
+Walck
+Waggener
+Vosburg
+Vita
+Villela
+Vercher
+Venters
+Vanscyoc
+Vandyne
+Valenza
+Utt
+Urick
+Ungar
+Ulm
+Tumlin
+Tsao
+Tryon
+Trudel
+Treiber
+Tow
+Tober
+Tipler
+Tillson
+Tiedemann
+Thornley
+Tetrault
+Temme
+Tarrance
+Tackitt
+Sykora
+Sweetman
+Swatzell
+Sutliff
+Suhr
+Sturtz
+Strub
+Strayhorn
+Stormer
+Steveson
+Stengel
+Steinfeldt
+Spiro
+Spieker
+Speth
+Spero
+Soza
+Souliere
+Soucie
+Snedeker
+Slifer
+Skillings
+Situ
+Siniard
+Simeon
+Signorelli
+Siggers
+Shultis
+Shrewsbury
+Shippee
+Shimp
+Sherron
+Shepler
+Sharpless
+Shadrick
+Severt
+Severs
+Semon
+Semmes
+Seiter
+Segers
+Sclafani
+Sciortino
+Schroyer
+Schrack
+Schoenberg
+Schober
+Scheidt
+Scheele
+Satter
+Sartori
+Sarris
+Sarratt
+Salvaggio
+Saladino
+Sakamoto
+Saine
+Ryman
+Rumley
+Ruggerio
+Rucks
+Roughton
+Room
+Robards
+Ricca
+Rexroad
+Resler
+Reny
+Rentschler
+Redrick
+Redick
+Reagle
+Raymo
+Rape
+Raker
+Racette
+Pyburn
+Pritt
+Presson
+Pressman
+Pough
+Plain
+Pisani
+Perz
+Perras
+Pelzer
+Pedrosa
+Palos
+Palmisano
+Paille
+Orem
+Orbison
+Oliveros
+Nourse
+Nordquist
+Newbury
+Nelligan
+Nawrocki
+Myler
+Mumaw
+Morphis
+Moldenhauer
+Miyashiro
+Mignone
+Mickelsen
+Michalec
+Mesta
+Mcree
+Mcqueary
+Mcninch
+Mcneilly
+Mclelland
+Mclawhorn
+Mcgreevy
+Mcconkey
+Mattes
+Maselli
+Marten
+Mart
+Marcucci
+Manseau
+Manjarrez
+Malbrough
+Machin
+Mabie
+Lynde
+Lykes
+Lueras
+Lokken
+Loken
+Linzy
+Lillis
+Lilienthal
+Levey
+Legler
+Leedom
+Lebowitz
+Lazzaro
+Larabee
+Lapinski
+Langner
+Langenfeld
+Lampkins
+Lamotte
+Lambright
+Lagarde
+Ladouceur
+Labrador
+Labounty
+Lablanc
+Laberge
+Kyte
+Kroon
+Kron
+Kraker
+Kouba
+Kirwin
+Kincer
+Kimbler
+Kegler
+Keach
+Katzman
+Katzer
+Kalman
+Journey
+Jimmerson
+Jenning
+Janus
+Iacovelli
+Hust
+Huson
+Husby
+Humphery
+Hufnagel
+Honig
+Holsey
+Holoman
+Hohl
+Hogge
+Hinderliter
+Hildebrant
+Hick
+Hey
+Hemby
+Helle
+Heintzelman
+Heidrick
+Hearon
+Heap
+Hazelip
+Hauk
+Hasbrouck
+Harton
+Hartin
+Harpster
+Hansley
+Hanchett
+Haar
+Guthridge
+Gulbranson
+Guill
+Guerrera
+Grund
+Grosvenor
+Grist
+Grell
+Grear
+Granberry
+Gonser
+Giunta
+Giuliani
+Gillon
+Gillmore
+Gillan
+Gibbon
+Gettys
+Gelb
+Gano
+Galliher
+Fullen
+Frese
+Frates
+Foxwell
+Fleishman
+Fleener
+Fielden
+Ferrera
+Feng
+Fells
+Feemster
+Fauntleroy
+Fails
+Evatt
+Espy
+Eno
+Emmerich
+Edwin
+Edler
+Eastham
+Dunavant
+Duca
+Drinnon
+Dowe
+Dorgan
+Dollinger
+Divers
+Dipalma
+Difranco
+Dietrick
+Denzer
+Demarest
+Delee
+Delariva
+Delany
+Decesare
+Debellis
+Deavers
+Deardorff
+Dawe
+Darosa
+Darley
+Dalzell
+Dahlen
+Curto
+Cupps
+Cunniff
+Cude
+Crivello
+Cripps
+Cresswell
+Cousar
+Cotta
+Compo
+Colorado
+Clyne
+Clayson
+Cearley
+Catania
+Carini
+Cargo
+Cantero
+Cali
+Buttrey
+Buttler
+Burpee
+Bulkley
+Buitron
+Buda
+Bublitz
+Bryer
+Bryden
+Brouillette
+Brott
+Brookman
+Bronk
+Breshears
+Brennen
+Brannum
+Brandl
+Braman
+Bracewell
+Boyter
+Bomberger
+Bold
+Bogen
+Boeding
+Bob
+Blauvelt
+Blandford
+Bigger
+Biermann
+Bielecki
+Bibby
+Berthold
+Berkman
+Belvin
+Bellomy
+Beland
+Behne
+Beecham
+Becher
+Beams
+Bax
+Bassham
+Barret
+Baley
+Bacchus
+Auxier
+Atkison
+Ary
+Arocha
+Arechiga
+Anspach
+An
+Algarin
+Alcott
+Alberty
+Ager
+Adolph
+Ackman
+Abdul
+Abdallah
+Zwick
+Ziemer
+Zastrow
+Zajicek
+Yokum
+Yokley
+Wittrock
+Winebarger
+Wilker
+Wilham
+Whitham
+Wetzler
+Westling
+Westbury
+Wendler
+Wellborn
+Weitzman
+Weitz
+Weight
+Wallner
+Waldroup
+Vrabel
+Vowels
+Volker
+Vitiello
+Visconti
+Villicana
+Vibbert
+Vesey
+Vannatter
+Vangilder
+Vandervort
+Vandegrift
+Vanalstyne
+Vallecillo
+Usrey
+Tynan
+Turpen
+Tuller
+Trisler
+Townson
+Tillmon
+Threlkeld
+Thornell
+Terrio
+Taunton
+Tarry
+Tardy
+Swoboda
+Swihart
+Sustaita
+Suitt
+Stuber
+Strine
+Stookey
+Stmartin
+Stiger
+Stainbrook
+Solem
+Smail
+Sligh
+Siple
+Sieben
+Shumake
+Shriner
+Showman
+Shiner
+Sheen
+Sheckler
+Seim
+Secrist
+Scoggin
+Schultheis
+Schmalz
+Schendel
+Schacher
+Savard
+Saulter
+Santillanes
+Sandiford
+Sande
+Salzer
+Salvato
+Saltz
+Sakai
+Ryckman
+Ryant
+Ruck
+Ronald
+Rocker
+Rittenberry
+Ristau
+Risk
+Richart
+Rhynes
+Reyer
+Reulet
+Reser
+Redington
+Reddington
+Rebello
+Reasor
+Raftery
+Rabago
+Raasch
+Quintanar
+Pylant
+Purington
+Provencal
+Prom
+Prioleau
+Prestwood
+Pothier
+Popa
+Polster
+Politte
+Poffenberger
+Pinner
+Pietrzak
+Pettie
+Penaflor
+Pellot
+Pellham
+Paylor
+Payeur
+Papas
+Paik
+Oyola
+Osbourn
+Orzechowski
+Oppenheimer
+Olesen
+Oja
+Ohl
+Nuckolls
+Nordberg
+Noonkester
+Nold
+Nitta
+Niblett
+Neuhaus
+Nesler
+Ned
+Nanney
+Myrie
+Mutch
+Motto
+Mosquera
+Morena
+Montalto
+Montagna
+Mizelle
+Mincy
+Millikan
+Millay
+Miler
+Milbourn
+Mikels
+Migues
+Miesner
+Mershon
+Merrow
+Merlin
+Melia
+Meigs
+Mealey
+Mcraney
+Mcmartin
+Mclachlan
+Mcgeehan
+Mcferren
+Mcdole
+Mccaulley
+Mcanulty
+Maziarz
+Maul
+Mateer
+Martinsen
+Marson
+Mariotti
+Manna
+Mang
+Mance
+Malbon
+Mah
+Magnusson
+Maclachlan
+Macek
+Lurie
+Luc
+Lown
+Loranger
+Lonon
+Lisenby
+Linsley
+Linger
+Lenk
+Leavens
+Learned
+Lauritzen
+Lathem
+Lashbrook
+Landman
+Lamarche
+Lamantia
+Laguerre
+Lagrange
+Kogan
+Klingbeil
+Kist
+Kimpel
+Kime
+Kier
+Kerfoot
+Kennamer
+Kellems
+Kammer
+Kamen
+Jess
+Jepsen
+Jarnigan
+Isler
+Ishee
+Isabel
+Hux
+Hungate
+Hummell
+Hultgren
+Huffaker
+Hruby
+Hover
+Hornick
+Hooser
+Hooley
+Hoggan
+Hirano
+Hilley
+Higham
+Heuser
+Henrickson
+Henegar
+Hellwig
+Heide
+Hedley
+Hasegawa
+Hartt
+Hambright
+Halfacre
+Hafley
+Guion
+Guinan
+Grunwald
+Grothe
+Gries
+Greaney
+Granda
+Grabill
+Gothard
+Gossman
+Gosser
+Gossard
+Gosha
+Goldner
+Gobin
+Gloss
+Ginyard
+Gilkes
+Gilden
+Gerson
+Gephart
+Gengler
+Gautier
+Gassett
+Garon
+Gandhi
+Galusha
+Gallager
+Galdamez
+Fulmore
+Fritsche
+Fowles
+Foutch
+Forward
+Footman
+Fludd
+Flakes
+Ferriera
+Ferrero
+Ferreri
+Fenimore
+Fegley
+Fegan
+Fearn
+Farrier
+Fansler
+Fane
+Falzone
+Fairweather
+Etherton
+Elsberry
+Dykema
+Duppstadt
+Dunnam
+Dunklin
+Duet
+Due
+Dudgeon
+Dubuc
+Doxey
+Dory
+Donmoyer
+Dodgen
+Disanto
+Dingler
+Dimattia
+Dilday
+Digennaro
+Diedrich
+Derossett
+Deputy
+Depp
+Demasi
+Degraffenreid
+Deakins
+Deady
+Davin
+Daigre
+Daddario
+Czerwinski
+Cullens
+Cubbage
+Cracraft
+Constance
+Comes
+Combest
+Coletti
+Coghill
+Clerk
+Claybrooks
+Class
+Christofferse
+Chiesa
+Chason
+Chamorro
+Cessna
+Celentano
+Cayer
+Carolan
+Carnegie
+Capetillo
+Callier
+Cadogan
+Caba
+Byrom
+Byrns
+Burrowes
+Burket
+Burdge
+Burbage
+Bukowski
+Buchholtz
+Brunt
+Brungardt
+Brunetti
+Brumbelow
+Brugger
+Broadhurst
+Brigance
+Brandow
+Bouknight
+Bottorff
+Bottomley
+Bosarge
+Borger
+Bona
+Bombardier
+Bologna
+Boggan
+Blumer
+Blecha
+Birney
+Birkland
+Betances
+Beran
+Benny
+Benes
+Belin
+Belgrave
+Bealer
+Bauch
+Bath
+Bashir
+Bartow
+Baro
+Barnhouse
+Barile
+Ballweg
+Baisley
+Bains
+Baehr
+Badilla
+Bachus
+Bacher
+Bachelder
+Auzenne
+Aten
+Astle
+Allis
+Agarwal
+Adger
+Adamek
+Ziolkowski
+Zinke
+Zazueta
+Zamorano
+Younkin
+Won
+Wittig
+Witman
+Winsett
+Winkles
+Wiedman
+Whitner
+Whitcher
+Wetherby
+Westra
+Westhoff
+Wehrle
+Wee
+Wagaman
+Voris
+Vicknair
+Vegas
+Veasley
+Vaugh
+Vanish
+Vanderburg
+Valletta
+Tunney
+Trumbo
+Truluck
+Trueman
+Truby
+Trombly
+Trojan
+Tourville
+Tostado
+Tone
+Titcomb
+Timpson
+Tignor
+Thrush
+Thresher
+Thiede
+Tews
+Tamplin
+Taff
+Tacker
+Syverson
+Sylvestre
+Summerall
+Stumbaugh
+Strouth
+Straker
+Stradford
+Stoney
+Stokley
+Steinhoff
+Steinberger
+Stairs
+Spigner
+Soltero
+Snively
+Sletten
+Sinkler
+Sinegal
+Simoes
+Siller
+Sigel
+Shoe
+Shire
+Shinkle
+Shellman
+Sheller
+Sheats
+Sharer
+Selvage
+Sedlak
+Sea
+Schriver
+Schimke
+Scheuerman
+Schanz
+Savory
+Saulters
+Sauers
+Sais
+Rusin
+Rumfelt
+Ruhland
+Rozar
+Rosborough
+Ronning
+Rolph
+Roloff
+Rogue
+Robie
+Riviera
+Rimer
+Riehle
+Ricco
+Rhein
+Retzlaff
+Reisman
+Reimann
+Re
+Rayes
+Raub
+Raminez
+Quesinberry
+Pua
+Procopio
+Priolo
+Printz
+Prewett
+Preas
+Prahl
+Portugal
+Poovey
+Ploof
+Platz
+Plaisted
+Pinzon
+Pineiro
+Pickney
+Petrovich
+Perl
+Pehrson
+Peets
+Pavon
+Pautz
+Pascarella
+Paras
+Paolini
+Pals
+Pafford
+Oyer
+Ovellette
+Outten
+Outen
+Ours
+Orduna
+Odriscoll
+Oberlin
+Nosal
+Niven
+Nisbett
+Nevers
+Nathanson
+Mule
+Mukai
+Mozee
+Mowers
+Motyka
+Morency
+Montford
+Mollica
+Molden
+Mitten
+Miser
+Mina
+Millender
+Midgette
+Messerly
+Melendy
+Meisel
+Meidinger
+Meany
+Mcnitt
+Mcnemar
+Mcmakin
+Mcgaugh
+Mccaa
+Mauriello
+Maudlin
+Matzke
+Mattia
+Matteo
+Matsumura
+Masuda
+Mangels
+Maloof
+Malizia
+Mahmoud
+Maglione
+Maddix
+Lucchesi
+Lochner
+Linquist
+Lino
+Lietz
+Leventhal
+Leopard
+Lemanski
+Leiser
+Laury
+Lauber
+Lamberth
+Kuss
+Kung
+Kulik
+Kuiper
+Krout
+Kotter
+Kort
+Kohlmeier
+Koffler
+Koeller
+Knipe
+Knauss
+Kleiber
+Kissee
+Kirst
+Kirch
+Kilgo
+Kerlin
+Kellison
+Kehl
+Kalb
+Jorden
+Jantzen
+Jamar
+Inabinet
+Ikard
+Husman
+Hunsberger
+Hundt
+Hucks
+Houtz
+Houseknecht
+Hoots
+Hogsett
+Hogans
+Hintze
+Hession
+Henault
+Hemming
+Helsley
+Heinen
+Heffington
+Heberling
+Heasley
+Heal
+Hazley
+Hazeltine
+Hayton
+Hayse
+Hawke
+Haston
+Harward
+Harvard
+Harrow
+Hanneman
+Hafford
+Hadnot
+Guerro
+Graig
+Grahm
+Gowins
+Gordillo
+Goosby
+Glatt
+Gibbens
+Ghent
+Gerrard
+Germann
+Geil
+Gebo
+Gean
+Garling
+Gardenhire
+Garbutt
+Gagner
+Furguson
+Funchess
+Fujiwara
+Fujita
+Friley
+Frigo
+Forshee
+Folkes
+Filler
+Fernald
+Ferber
+Feingold
+Favorite
+Faul
+Farrelly
+Fairbank
+Failla
+Estelle
+Espey
+Eshleman
+Ertl
+Erhart
+Erhardt
+Erbe
+Elsea
+Ells
+Ellman
+Eisenhart
+Ehmann
+Earnhardt
+Duplantis
+Dulac
+Ducote
+Draves
+Dosch
+Dolce
+Divito
+Ditch
+Dimauro
+Derringer
+Demeo
+Demartini
+Delima
+Dehner
+Degen
+Defrancisco
+Defoor
+Dedeaux
+Debnam
+Cypert
+Cutrer
+Cusumano
+Custis
+Croker
+Courtois
+Costantino
+Cormack
+Corbeil
+Copher
+Conlan
+Conkling
+Cogdell
+Cilley
+Chapdelaine
+Cendejas
+Castiglia
+Cassette
+Cashin
+Carstensen
+Carol
+Caprio
+Calcote
+Calaway
+Byfield
+Butner
+Bushway
+Burritt
+Browner
+Brobst
+Briner
+Brighton
+Bridger
+Brickley
+Brendel
+Bratten
+Bratt
+Brainerd
+Brackman
+Bowne
+Bouck
+Borunda
+Bordner
+Bonenfant
+Boer
+Boehmer
+Bodiford
+Bleau
+Blankinship
+Blane
+Blaha
+Bitting
+Bissonette
+Bigby
+Bibeau
+Beverage
+Bermudes
+Berke
+Bergevin
+Bergerson
+Bendel
+Belville
+Bechard
+Bearce
+Beadles
+Batz
+Bartlow
+Barren
+Ayoub
+Avans
+Aumiller
+Arviso
+Arpin
+Arnwine
+Armwood
+Arent
+Arehart
+Arcand
+Antle
+Ambrosino
+Alongi
+Alm
+Allshouse
+Ahart
+Aguon
+Ziebarth
+Zeledon
+Zakrzewski
+Yuhas
+Yingst
+Yedinak
+Wommack
+Winnett
+Wingler
+Wilcoxen
+Whitmarsh
+Whistler
+Wayt
+Watley
+Wasser
+Warkentin
+Voll
+Vogelsang
+Voegele
+Vivanco
+Vinton
+Villafane
+Viles
+Versace
+Ver
+Venne
+Vanwagoner
+Vanwagenen
+Vanleuven
+Vanauken
+Uselton
+Uren
+Trumbauer
+Tritt
+Treadaway
+Tozier
+Tope
+Tomczak
+Tomberlin
+Tomasini
+Tollett
+Toller
+Titsworth
+Tirrell
+Tilly
+Tavera
+Tarnowski
+Tanouye
+Tall
+Swarthout
+Sutera
+Surette
+Styers
+Styer
+Stipe
+Stickland
+Steve
+Stembridge
+Stearn
+Starkes
+Stanberry
+Stahr
+Spino
+Spicher
+Sperber
+Speece
+Soo
+Sonntag
+Sneller
+Smalling
+Slowik
+Slocumb
+Sliva
+Slemp
+Slama
+Sitz
+Sisto
+Sisemore
+Sindelar
+Shipton
+Shillings
+Sheeley
+Sharber
+Shaddix
+Severns
+Severino
+Sever
+Sensabaugh
+Seder
+Seawell
+Seamons
+Schrantz
+Schooler
+Scheffer
+Scheerer
+Scalia
+Saum
+Santibanez
+Sano
+Sanjuan
+Sampley
+Sailer
+Sabella
+Sabbagh
+Royall
+Rottman
+Rivenbark
+Rikard
+Ricketson
+Rickel
+Rethman
+Reily
+Reddin
+Reasoner
+Reade
+Rast
+Ranallo
+Rana
+Quintal
+Pung
+Pucci
+Proto
+Prosperie
+Prim
+Preusser
+Preslar
+Powley
+Postma
+Pinnix
+Pilla
+Pietsch
+Pickerel
+Pica
+Pharris
+Petway
+Petillo
+Perin
+Pereda
+Pennypacker
+Pennebaker
+Pedrick
+Patin
+Patchell
+Parodi
+Parman
+Pantano
+Padua
+Padro
+Osterhout
+Orner
+Opp
+Olivar
+Ohlson
+Odonoghue
+Oceguera
+Oberry
+Novello
+Noguera
+Newquist
+Newcombe
+Neihoff
+Nehring
+Nees
+Nebeker
+Nau
+Mundo
+Mullenix
+Morrisey
+Moronta
+Morillo
+Morefield
+Mongillo
+Molino
+Minto
+Midgley
+Michie
+Menzies
+Medved
+Mechling
+Mealy
+Mcshan
+Mcquaig
+Mcnees
+Mcglade
+Mcgarity
+Mcgahey
+Mcduff
+Mayweather
+Mastropietro
+Masten
+Maranto
+Maniscalco
+Maize
+Mahmood
+Maddocks
+Maday
+Macha
+Maag
+Luken
+Lopp
+Lolley
+Llanas
+Litz
+Litherland
+Lindenberg
+Lieu
+Letcher
+Lentini
+Lemelle
+Leet
+Lecuyer
+Leber
+Laursen
+Latch
+Larrick
+Lantigua
+Langlinais
+Lalli
+Lafever
+Labat
+Labadie
+Kurt
+Krogman
+Kohut
+Knarr
+Klimas
+Klar
+Kittelson
+Kirschbaum
+Kintzel
+Kincannon
+Kimmell
+Killgore
+Kettner
+Kelsch
+Karle
+Kapoor
+Johansson
+Jock
+Jenkinson
+Janney
+Isabelle
+Iraheta
+Insley
+Hyslop
+Hy
+Human
+Huckstep
+Holleran
+Hoerr
+Hinze
+Hinnenkamp
+Hilger
+Higgin
+Hicklin
+Heroux
+Henkle
+Helfer
+Heikkinen
+Heckstall
+Heckler
+Heavener
+Haydel
+Haveman
+Haubert
+Harrop
+Harnois
+Hansard
+Hanover
+Hammitt
+Haliburton
+Haefner
+Hadsell
+Haakenson
+Guynn
+Guizar
+Grout
+Grosz
+Goo
+Gomer
+Golla
+Godby
+Glanz
+Glancy
+Givan
+Giesen
+Gerst
+Gayman
+Garraway
+Gabor
+Furness
+Frisk
+Fremont
+Frary
+Forand
+Fessenden
+Ferrigno
+Fearon
+Favreau
+Faulks
+Falbo
+Ewen
+Everton
+Eurich
+Etchison
+Esterly
+Entwistle
+Ellingsworth
+Elders
+Ek
+Eisenbarth
+Edelson
+Eckel
+Earnshaw
+Dunneback
+Doyal
+Donnellan
+Dolin
+Dibiase
+Deschenes
+Dermody
+Denmark
+Degregorio
+Darnall
+Dant
+Dansereau
+Danaher
+Dammann
+Dames
+Czarnecki
+Cuyler
+Custard
+Cummingham
+Cuffie
+Cuffee
+Cudney
+Cuadra
+Crigler
+Creger
+Coughlan
+Corvin
+Cortright
+Corchado
+Connery
+Conforti
+Condron
+Colosimo
+Colclough
+Cola
+Cohee
+Claire
+Ciotti
+Chill
+Chien
+Check
+Chacko
+Cevallos
+Cavitt
+Cavins
+Castagna
+Cashwell
+Carrozza
+Carrara
+Capra
+Campas
+Callas
+Caison
+Cai
+Caggiano
+Cabot
+Bynoe
+Buswell
+Burpo
+Burnam
+Burges
+Buerger
+Buelow
+Bueche
+Buckle
+Bruni
+Brummitt
+Brodersen
+Briese
+Breit
+Brakebill
+Braatz
+Boyers
+Boughner
+Borror
+Borquez
+Bonelli
+Bohner
+Blaze
+Blaker
+Blackmer
+Bissette
+Bibbins
+Bhatt
+Bhatia
+Bessler
+Bergh
+Beresford
+Bensen
+Benningfield
+Benito
+Bellantoni
+Behler
+Beehler
+Beazley
+Beauchesne
+Bargo
+Bannerman
+Baltes
+Balog
+Ballantyne
+Bad
+Axelson
+Apgar
+Aoki
+Anstett
+Alejos
+Alcocer
+Albury
+Aichele
+Ahl
+Ackles
+Zerangue
+Zehner
+Zank
+Zacarias
+Youngberg
+Yorke
+Yarbro
+Xie
+Wydra
+Worthley
+Wolbert
+Wittmer
+Witherington
+Wishart
+Wire
+Winnie
+Winkleman
+Willilams
+Willer
+Wiedeman
+Whittingham
+Whitbeck
+Whetsel
+Wheless
+Westerberg
+Welcher
+Wegman
+Waterfield
+Wasinger
+Warfel
+Wannamaker
+Walborn
+Wada
+Vogl
+Vizcarrondo
+Vitela
+Villeda
+Veras
+Venuti
+Veney
+Ulrey
+Uhlig
+Turcios
+Tremper
+Torian
+Torbett
+Thrailkill
+Terrones
+Teitelbaum
+Teems
+Tay
+Swoope
+Sunseri
+Stutes
+Stthomas
+Strohm
+Stroble
+Striegel
+Streicher
+Stodola
+Stinchcomb
+Steves
+Steppe
+Stem
+Steller
+Staudt
+Starner
+Stamant
+Stam
+Stackpole
+Sprankle
+Speciale
+Spahr
+Sowders
+Sova
+Soluri
+Soderlund
+Slinkard
+Skates
+Sjogren
+Sirianni
+Siewert
+Sickels
+Sica
+Shugart
+Shoults
+Shive
+Shimer
+Shier
+Shield
+Shepley
+Sheeran
+Sharper
+Sevin
+Severe
+Seto
+Segundo
+Sedlacek
+Scuderi
+Schurman
+Schuelke
+Scholten
+Schlater
+Schisler
+Schiefelbein
+Schalk
+Sanon
+Sae
+Sabala
+Ruyle
+Ruybal
+Ruf
+Rueb
+Rowsey
+Rosol
+Rocheleau
+Rishel
+Rippey
+Ringgold
+Rieves
+Ridinger
+Rew
+Retherford
+Rempe
+Reith
+Rafter
+Raffaele
+Quinto
+Putz
+Purdom
+Puls
+Pulaski
+Propp
+Principato
+Preiss
+Prada
+Polansky
+Poch
+Plath
+Pittard
+Pinnock
+Pfarr
+Pfannenstiel
+Penniman
+Pauling
+Patchen
+Paschke
+Parkey
+Pando
+Overly
+Ouimet
+Ottman
+Otter
+Ostlund
+Ormiston
+Occhipinti
+Nowacki
+Norred
+Noack
+Nishida
+Nilles
+Nicodemus
+Neth
+Nealey
+Myricks
+Murff
+Mungia
+Mullet
+Motsinger
+Moscato
+Mort
+Morado
+Moors
+Monnier
+Molyneux
+Modzelewski
+Miura
+Minich
+Militello
+Milbrandt
+Michalik
+Meserve
+Merle
+Mendivil
+Melara
+Meadow
+Mcnish
+Mcelhannon
+Mccroy
+Mccrady
+Mazzella
+Maule
+Mattera
+Mathena
+Matas
+Mass
+Mascorro
+Marone
+Marinello
+Marguez
+Marcell
+Manwaring
+Manhart
+Mangano
+Maggi
+Lymon
+Luter
+Luse
+Lukasik
+Luiz
+Ludlum
+Luczak
+Lowenthal
+Lossett
+Lorentzen
+Loredo
+Longworth
+Lomanto
+Lisi
+Lish
+Lipsky
+Linck
+Liedtke
+Levering
+Lessman
+Lemond
+Lembo
+Ledonne
+Leatham
+Laufer
+Lanphear
+Langlais
+Lando
+Lamphear
+Lamberton
+Lafon
+Lade
+Lacross
+Kyzer
+Krok
+Kring
+Krell
+Krehbiel
+Kratochvil
+Krach
+Kovar
+Kostka
+Knudtson
+Knaack
+Kliebert
+Klahn
+Kirkley
+Kimzey
+Kettle
+Kerrick
+Kennerson
+Keesler
+Karlin
+Kan
+Jenny
+Janousek
+Jan
+Imel
+Icenhour
+Hyler
+Hunger
+Hudock
+Houpt
+Hopping
+Hoops
+Holquin
+Holiman
+Holahan
+Hodapp
+Hires
+Hillen
+Hickmon
+Hersom
+Henrich
+Helvey
+Heidt
+Heideman
+Hedstrom
+Hedin
+Hebron
+Hayter
+Harn
+Hardage
+Harbor
+Halsted
+Hahne
+Hagemann
+Guzik
+Guel
+Groesbeck
+Gritton
+Grego
+Graziani
+Grasty
+Graney
+Gouin
+Gossage
+Golston
+Goheen
+Godina
+Glade
+Giorgi
+Giambrone
+Gerrity
+Gerrish
+Gero
+Gerling
+Gaulke
+Garlick
+Galiano
+Gaiter
+Gahagan
+Gagnier
+Friddle
+Fredericksen
+Franqui
+Follansbee
+Foerster
+Flury
+Fitzmaurice
+Fiorini
+Finlayson
+Fiecke
+Fickes
+Fichter
+Ferron
+Ferdinand
+Farrel
+Fackler
+Eyman
+Escarcega
+Errico
+Erler
+Erby
+Engman
+Engelmann
+Elsass
+Elliston
+Eddleman
+Eadie
+Dummer
+Drost
+Dorrough
+Dorrance
+Doolan
+Donalson
+Domenico
+Ditullio
+Dittmar
+Dishon
+Dionisio
+Dike
+Devinney
+Desir
+Deschamp
+Derrickson
+Delamora
+Deitch
+Dechant
+Dave
+Danek
+Dahmen
+Curci
+Cudjoe
+Crumble
+Croxton
+Creasman
+Craney
+Crader
+Cowling
+Coulston
+Cortina
+Corlew
+Corl
+Copland
+Convery
+Cohrs
+Clune
+Clausing
+Cipriani
+Cinnamon
+Cianciolo
+Chubb
+Chittum
+Chenard
+Charlesworth
+Charlebois
+Champine
+Chamlee
+Chagoya
+Casselman
+Cardello
+Capasso
+Cannella
+Calderwood
+Byford
+Buttars
+Bushee
+Burrage
+Buentello
+Brzozowski
+Bryner
+Brumit
+Brookover
+Bronner
+Bromberg
+Brixey
+Brinn
+Briganti
+Bremner
+Brawn
+Branscome
+Brannigan
+Bradsher
+Bozek
+Boulay
+Bormann
+Bongiorno
+Bollin
+Bohler
+Bogert
+Bodenhamer
+Blose
+Blind
+Bivona
+Bitter
+Billips
+Bibler
+Benfer
+Benedetti
+Belue
+Bellanger
+Belford
+Behn
+Beerman
+Barnhardt
+Baltzell
+Balling
+Balducci
+Bainter
+Babineau
+Babich
+Baade
+Attwood
+Asmus
+Asaro
+Artiaga
+April
+Applebaum
+Ang
+Anding
+Amar
+Amaker
+Allsup
+Alligood
+Alers
+Agin
+Agar
+Achenbach
+Abramowitz
+Abbas
+Aasen
+Zehnder
+Yopp
+Yelle
+Yeldell
+Wynter
+Woodmansee
+Wooding
+Woll
+Winborne
+Willsey
+Willeford
+Widger
+Whiten
+Whitchurch
+Whang
+Wen
+Weissinger
+Weinman
+Weingartner
+Weidler
+Waltrip
+Walt
+Wagar
+Wafford
+Vitagliano
+Villalvazo
+Villacorta
+Vigna
+Vickrey
+Vicini
+Ventimiglia
+Vandenbosch
+Valvo
+Valazquez
+Utsey
+Urbaniak
+Unzueta
+Trombetta
+Trevizo
+Trembley
+Tremaine
+Traverso
+Tores
+Tolan
+Tillison
+Tietjen
+Tee
+Teachout
+Taube
+Tatham
+Tarwater
+Tarbell
+Sydow
+Sy
+Swims
+Swader
+Striplin
+Stops
+Stoltenberg
+Steinhauer
+Steil
+Steigerwald
+Starkweather
+Stallman
+Squier
+Sparacino
+Span
+Spadafora
+Shiflet
+Shibata
+Shevlin
+Sherrick
+Shake
+Sessums
+Servais
+Senters
+Seevers
+Seelye
+Searfoss
+Seabrooks
+Scoles
+Schwager
+Schrom
+Schmeltzer
+Scheffel
+Sax
+Sawin
+Saterfiel
+Sardina
+Sanroman
+Sane
+Sandin
+Salamanca
+Saladin
+Sak
+Sabia
+Rustin
+Rushin
+Ruley
+Rueter
+Row
+Rotter
+Rosenzweig
+Roles
+Rohe
+Roder
+Rockey
+Ro
+Riter
+Rieth
+Ried
+Riding
+Riddles
+Ridder
+Rennick
+Remmers
+Remer
+Relyea
+Reilley
+Reder
+Rasheed
+Rakowski
+Rabin
+Queener
+Pursel
+Prue
+Prowell
+Pritts
+Primo
+Presler
+Pouncy
+Porche
+Porcaro
+Pollman
+Pleas
+Planas
+Pinkley
+Pinegar
+Pilger
+Philson
+Petties
+Perrodin
+Pendergrast
+Patao
+Pasternak
+Passarelli
+Pasko
+Parshall
+Panos
+Panella
+Palombo
+Padillo
+Oyama
+Overlock
+Overbeck
+Otterson
+Orrell
+Ornellas
+Opitz
+Okelly
+Officer
+Obando
+Noggle
+Nicosia
+Netto
+Negrin
+Natali
+Nakayama
+Nagao
+Nadel
+Musial
+Murrill
+Murrah
+Munsch
+Mucci
+Mrozek
+Moyes
+Mowrer
+Moris
+Morais
+Moorhouse
+Monico
+Mone
+Mondy
+Moncayo
+Mole
+Miltenberger
+Milsap
+Milone
+Millikin
+Milardo
+Mika
+Micheals
+Micco
+Meyerson
+Mericle
+Mendell
+Meinhardt
+Meachum
+Mcleroy
+Mcgray
+Mcgonigal
+Maultsby
+Matis
+Matheney
+Matamoros
+Marro
+Marcil
+Marcial
+Mantz
+Mannings
+Maltby
+Malchow
+Maiorano
+Mahn
+Mahlum
+Maglio
+Mae
+Maberry
+Lustig
+Luellen
+Longwell
+Longenecker
+Lofland
+Locascio
+Linney
+Linneman
+Lighty
+Levell
+Levay
+Lenahan
+Lemen
+Lehto
+Lebaron
+Lanctot
+Lamy
+Lainez
+Laffoon
+Labombard
+Kujawski
+Kroger
+Kreutzer
+Korhonen
+Kondo
+Kollman
+Kohan
+Kogut
+Knaus
+Kivi
+Kittel
+Kinner
+Kindig
+Kindel
+Kiesel
+Kidney
+Kibby
+Khang
+Kettler
+Ketterer
+Kepner
+Kelliher
+Keenum
+Kanode
+Kail
+July
+Juhasz
+Jowett
+Jolicoeur
+Jeon
+Iser
+Ingrassia
+Imai
+Hutchcraft
+Humiston
+Hulings
+Hukill
+Huizenga
+Hugley
+Huddle
+Hose
+Hornyak
+Hodder
+Hisle
+Hillenbrand
+Hille
+Higuchi
+Hertzler
+Herdon
+Heppner
+Hepp
+Heitmann
+Heckart
+Hazlewood
+Hayles
+Hayek
+Hawthorn
+Hawkin
+Haugland
+Hasler
+Harbuck
+Happel
+Hambly
+Hambleton
+Hagaman
+Guzzi
+Gullette
+Guinyard
+Grogg
+Grise
+Griffing
+Goto
+Gosney
+Goods
+Goley
+Goldblatt
+Gledhill
+Girton
+Giltner
+Gillock
+Gilham
+Gilfillan
+Giblin
+Gentner
+Gehlert
+Gehl
+Garten
+Garney
+Garlow
+Garett
+Galles
+Galeana
+Futral
+Fuhr
+Friedland
+Franson
+Fransen
+Foulds
+Follmer
+Foland
+Flax
+Flavin
+Firkins
+Fillion
+Figueredo
+Ferrill
+Fenster
+Fenley
+Fauver
+Farfan
+Factor
+Eustice
+Eppler
+Engelman
+Engelke
+Emmer
+Elzy
+Ellwood
+Ellerbee
+Elks
+Ehret
+Ebbert
+Durrah
+Dupras
+Dubuque
+Dragoo
+Donlon
+Dolloff
+Doi
+Dibella
+Derrico
+Demko
+Demar
+Darrington
+Czapla
+Crooker
+Creagh
+Cranor
+Craner
+Crafts
+Crabill
+Coyer
+Cowman
+Cowherd
+Cottone
+Costillo
+Coster
+Costas
+Cosenza
+Corker
+Collinson
+Coello
+Clingman
+Clingerman
+Claborn
+Citizen
+Chmura
+Chausse
+Chaudhry
+Chapell
+Chancy
+Cerrone
+Caves
+Caverly
+Caulkins
+Carn
+Campfield
+Campanelli
+Callaham
+Cadorette
+Butkovich
+Buske
+Burrier
+Burkley
+Bunyard
+Budge
+Buckelew
+Buchheit
+Broman
+Brescia
+Brasel
+Brain
+Boyster
+Booe
+Bonomo
+Bonnet
+Bondi
+Bohnsack
+Bobby
+Blomberg
+Blanford
+Bilderback
+Biggins
+Bently
+Behrends
+Beegle
+Bedoya
+Bechtol
+Beaubien
+Bayerl
+Baumgart
+Baumeister
+Barratt
+Barlowe
+Barkman
+Barbagallo
+Baldree
+Baine
+Bail
+Baggs
+Bacote
+Aylward
+Ashurst
+Arvidson
+Arthurs
+Arrieta
+Arrey
+Arreguin
+Arrant
+Arner
+Armor
+Arizmendi
+Anker
+Amis
+Amend
+Alphin
+Allbright
+Aikin
+Acres
+Zupan
+Zuchowski
+Zeolla
+Zanchez
+Zahradnik
+Zahler
+Younan
+Yeater
+Yearta
+Yarrington
+Yantis
+Woomer
+Wollard
+Wolfinger
+Woerner
+Witek
+Wishon
+Wisener
+Wingerter
+Willet
+Wilding
+Wiedemann
+Weisel
+Wedeking
+Weary
+Waybright
+Wardwell
+Walkins
+Waldorf
+Voth
+Voit
+Virden
+Viloria
+Villagran
+Vasta
+Vashon
+Vaquera
+Vantassell
+Vanderlinden
+Vandergrift
+Vancuren
+Valenta
+Underdahl
+Tyra
+Tygart
+Twining
+Twiford
+Turlington
+Tullius
+Tubman
+Trowell
+Trieu
+Transue
+Tousant
+Torgersen
+Tooker
+Tony
+Tome
+Toma
+Tocci
+Tippins
+Tinner
+Timlin
+Tillinghast
+Tidmore
+Teti
+Tedrick
+Tacey
+Swanberg
+Sunde
+Summitt
+Summerford
+Summa
+Sue
+Stratman
+Strandberg
+Storck
+Stober
+Steitz
+Stayer
+Stauber
+Staiger
+Sponaugle
+Spofford
+Sparano
+Spagnola
+Sokoloski
+Snay
+Slough
+Skowronski
+Sieck
+Shimkus
+Sheth
+Sherk
+Shankles
+Shakespeare
+Shahid
+Sevy
+Sergeant
+Senegal
+Seiden
+Seidell
+Searls
+Searight
+Schwalm
+Schug
+Schilke
+Schier
+Scheck
+Sawtelle
+Santore
+Santa
+Sanks
+Sandquist
+Sanden
+Saling
+Sabine
+Saathoff
+Ryberg
+Rustad
+Ruffing
+Rudnicki
+Ruane
+Rozzi
+Rowse
+Rosenau
+Rodes
+Risser
+Riggin
+Riess
+Riese
+Rhoten
+Reinecke
+Reigle
+Reichling
+Redner
+Rebelo
+Raynes
+Raimondi
+Rahe
+Rada
+Querry
+Quellette
+Pulsifer
+Prochnow
+Pretty
+Prato
+Poulton
+Poudrier
+Poll
+Policastro
+Polhemus
+Polasek
+Poissant
+Pohlmann
+Plotner
+Pitkin
+Pita
+Pio
+Pinkett
+Pilot
+Piekarski
+Pichon
+Philippe
+Pfau
+Petroff
+Petermann
+Peplinski
+Peller
+Pecinovsky
+Pearse
+Pattillo
+Patague
+Parlier
+Parenti
+Parchman
+Pane
+Paff
+Ota
+Ortner
+Oros
+Nolley
+Noakes
+Nigh
+Nicolosi
+Nicolay
+Newnam
+Netter
+Nass
+Napoles
+Nakata
+Nakamoto
+Muriel
+Muck
+Morlock
+Moraga
+Montilla
+Mongeau
+Molitor
+Mohney
+Mitchener
+Meyerhoff
+Medel
+Mcniff
+Mcmonagle
+Mcglown
+Mcglinchey
+Mcgarrity
+Mccright
+Mccorvey
+Mcconnel
+Mccargo
+Mazzei
+Matula
+Mastroianni
+Massingale
+Maring
+Maricle
+Marc
+Mans
+Mannon
+Mannix
+Manney
+Manger
+Manalo
+Malo
+Malan
+Mahony
+Madril
+Mackowiak
+Macko
+Macintosh
+Lurry
+Luczynski
+Lucke
+Lucarelli
+Luca
+Loud
+Lou
+Losee
+Lorence
+Loiacono
+Lohse
+Loder
+Lipari
+Linebarger
+Lindamood
+Limbaugh
+Letts
+Leleux
+Leep
+Leeder
+Leard
+Laxson
+Lawry
+Laverdiere
+Laughton
+Lastra
+Kurek
+Kriss
+Krishnan
+Kretschmer
+Krebsbach
+Kontos
+Knobel
+Knauf
+Klick
+Kleven
+Klawitter
+Kitchin
+Kirkendoll
+Kinkel
+Kingrey
+Kilbourn
+Kensinger
+Kennerly
+Kamin
+Justiniano
+Jurek
+Junkin
+Julia
+Judon
+Jordahl
+Jeanes
+Jarrells
+Jamal
+Iwamoto
+Isreal
+Ishida
+Ines
+Immel
+Iman
+Ihle
+Hyre
+Hurn
+Hunn
+Hultman
+Huffstetler
+Huffer
+Hubner
+Howey
+Horney
+Hooton
+Holts
+Holscher
+Holen
+Hoggatt
+Hilaire
+Herz
+Henne
+Helstrom
+Hellickson
+Heinlein
+Heckathorn
+Heckard
+Heather
+Heart
+Headlee
+Hauptman
+Haughey
+Hatt
+Harring
+Harford
+Hammill
+Hamed
+Halperin
+Haig
+Hagwood
+Hagstrom
+Gunnells
+Gundlach
+Guardiola
+Greeno
+Greenland
+Gonce
+Goldsby
+Gobel
+Gisi
+Gillins
+Gillie
+Germano
+Geibel
+Gauger
+Garriott
+Garbarino
+Gander
+Gajewski
+Funari
+Fullbright
+Fuell
+Fritzler
+Freshwater
+Freas
+Fortino
+Forbus
+Fonda
+Flohr
+Flemister
+Fisch
+Finks
+Fenstermaker
+Feldstein
+Faw
+Farhat
+Farah
+Fankhauser
+Fagg
+Fader
+Exline
+Emigh
+Eguia
+Edman
+Eckler
+Eastburn
+Dy
+Dunmore
+Dubuisson
+Dubinsky
+Drayer
+Doverspike
+Doubleday
+Doten
+Dorner
+Dolson
+Dohrmann
+Disla
+Direnzo
+Dipaola
+Dines
+Dickie
+Diblasi
+Dewolf
+Desanti
+Dennehy
+Demming
+Delker
+Decola
+Davilla
+Davids
+Daughtridge
+Darville
+Darland
+Danzy
+Dandy
+Dagenais
+Culotta
+Cruzado
+Crudup
+Croswell
+Coverdale
+Covelli
+Couts
+Corbell
+Coplan
+Coolbaugh
+Conyer
+Conlee
+Conigliaro
+Comiskey
+Coberly
+Clendening
+Clairmont
+Cienfuegos
+Chojnacki
+Chilcote
+Champney
+Cassara
+Casazza
+Casado
+Carew
+Carbin
+Carabajal
+Calcagni
+Cail
+Caddy
+Busbee
+Burts
+Burbridge
+Bunge
+Bundick
+Buhler
+Bucker
+Bucholtz
+Bruen
+Broce
+Brite
+Brignac
+Brierly
+Bridgman
+Braham
+Bradish
+Boyington
+Borjas
+Bonnie
+Bonn
+Bonhomme
+Bohlen
+Bogardus
+Bockelman
+Blick
+Blackerby
+Bizier
+Biro
+Binney
+Bertolini
+Bertin
+Berti
+Bert
+Bento
+Beno
+Belgarde
+Belding
+Beckel
+Becerril
+Bazaldua
+Bayes
+Bayard
+Barrus
+Barris
+Baros
+Bara
+Ballow
+Balboa
+Bakewell
+Baginski
+Badalamenti
+Backhaus
+Avilez
+Auvil
+Atteberry
+Ardon
+Anzaldua
+Anello
+Amsler
+Amo
+Ambrosio
+Althouse
+Alles
+Alix
+Alberti
+Alberson
+Aitchison
+Aguinaga
+Ziemann
+Zickefoose
+Zerr
+Zeh
+Zeck
+Zartman
+Zahm
+Zabriskie
+Yohn
+Yellowhair
+Yeaton
+Yarnall
+Yaple
+Wolski
+Wixon
+Winford
+Willner
+Willms
+Whitsitt
+Wheelwright
+Weyandt
+Wess
+Wengerd
+Weatherholtz
+Wattenbarger
+Walrath
+Walpole
+Waldrip
+Voges
+Violet
+Vinzant
+Viars
+Veres
+Veneziano
+Veillon
+Vawter
+Vaughns
+Vanwart
+Vanostrand
+Valiente
+Valderas
+Uhrig
+Tunison
+Tulloch
+Trostle
+Treaster
+Traywick
+Toye
+Tomson
+Tomasello
+Tomasek
+Tippit
+Tinajero
+Tift
+Tienda
+Thorington
+Thierry
+Thieme
+Thibeau
+Thakkar
+Tewell
+Test
+Telfer
+Sweetser
+Sum
+Stratford
+Stracener
+Stoke
+Stiverson
+Stelling
+Stefan
+Stavros
+Speaker
+Spatz
+Spagnoli
+Sorge
+Sober
+Slevin
+Slabaugh
+Simson
+Shupp
+Shoultz
+Shotts
+Shiroma
+Shetley
+Sherrow
+Sheffey
+Shawgo
+Shamburger
+Sester
+Segraves
+Seelig
+Seats
+Scioneaux
+Schwartzkopf
+Schwabe
+Scholes
+Schmuck
+Schluter
+Schlecht
+Schillaci
+Schildgen
+Schieber
+Schewe
+Schecter
+Scarpelli
+Scaglione
+Sautter
+Santelli
+Sandman
+Salmi
+Sabado
+Ryer
+Rydberg
+Ryba
+Rushford
+Running
+Runk
+Ruddick
+Rotondo
+Rote
+Rosenfield
+Roesner
+Rocchio
+Ritzer
+Rippel
+Rimes
+Riffel
+Richison
+Ribble
+Reynold
+Resh
+Rehn
+Ratti
+Rasor
+Rasnake
+Rappold
+Rando
+Radosevich
+Pulice
+Puff
+Prichett
+Pribble
+Poynor
+Plowden
+Pitzen
+Pittsley
+Pitter
+Pigeon
+Philyaw
+Philipps
+Petite
+Pestana
+Perro
+Perone
+Pera
+Peil
+Pedone
+Pawlowicz
+Pattee
+Parten
+Parlin
+Pariseau
+Paredez
+Pardon
+Panther
+Paek
+Pacifico
+Otts
+Ostrow
+Osornio
+Oslund
+Orso
+Ooten
+Onken
+Oniel
+Onan
+Ollison
+Ohlsen
+Ohlinger
+Odowd
+Niemiec
+Neubert
+Nembhard
+Neaves
+Neathery
+Nakasone
+Myerson
+Muto
+Muntz
+Munez
+Mumme
+Mumm
+Mujica
+Muise
+Muench
+Morriss
+Molock
+Mishoe
+Minier
+Metzgar
+Mero
+Meiser
+Meese
+Meals
+Mcsween
+Mcquire
+Mcquinn
+Mcpheeters
+Mckeller
+Mcilrath
+Mcgown
+Mcdavis
+Mccuen
+Mcclenton
+Maxham
+Matsui
+Marriner
+Marlette
+Mantle
+Mansur
+Mancino
+Maland
+Majka
+Maisch
+Maheux
+Madry
+Madriz
+Mackley
+Macke
+Lydick
+Lutterman
+Luppino
+Lundahl
+Lovingood
+Loudon
+Longmore
+Lippman
+Liefer
+Leveque
+Lescarbeau
+Lemmer
+Ledgerwood
+Lawver
+Lawrie
+Lattea
+Lasko
+Lahman
+Kulpa
+Kukowski
+Kukla
+Kubota
+Kubala
+Krizan
+Kriz
+Krikorian
+Kravetz
+Kramp
+Kowaleski
+Knobloch
+Klosterman
+Kloster
+Klepper
+Kirven
+Kinnaman
+Kinnaird
+Killam
+Kiesling
+Kesner
+Keebler
+Keagle
+Karls
+Kapinos
+Kantner
+Kaba
+Junious
+Jefferys
+Jacquet
+Izzi
+Ishii
+Irion
+Ifill
+Hyun
+Hotard
+Horman
+Hoppes
+Hopkin
+Hokanson
+Hoda
+Hocutt
+Hoaglin
+Hites
+Hirai
+Hindle
+Hinch
+Hilty
+Hild
+Hier
+Hickle
+Hibler
+Henrichs
+Hempstead
+Helmers
+Hellard
+Heims
+Heidler
+Hearst
+Hawbaker
+Hau
+Harkleroad
+Harari
+Hanney
+Hannaford
+Hamid
+Hamburger
+Haltom
+Hallford
+Guilliams
+Guerette
+Gryder
+Groseclose
+Groen
+Grimley
+Greenidge
+Greek
+Graffam
+Goucher
+Goodenough
+Goldsborough
+Goldie
+Gloster
+Glanton
+Gladson
+Gladding
+Ghee
+Gethers
+Gerstein
+Geesey
+Geddie
+Gayer
+Gaw
+Gaver
+Gauntt
+Gartland
+Garriga
+Garoutte
+Gao
+Gan
+Fronk
+Fritze
+Frenzel
+Forgione
+Fluitt
+Flinchbaugh
+Flach
+Fiorito
+Finan
+Finamore
+Fimbres
+Fillman
+File
+Figeroa
+Ficklin
+Feher
+Feddersen
+Fambro
+Fairbairn
+Eves
+Esperanza
+Escalona
+Elsey
+Eisenstein
+Ehrenberg
+Eargle
+Dress
+Drane
+Dorothy
+Doria
+Dogan
+Dively
+Dewolfe
+Dettman
+Desiderio
+Desch
+Dennen
+Denk
+Demaris
+Delsignore
+Dejarnette
+Deere
+Dedman
+Daws
+Dawn
+Dauphinais
+Danz
+Dantin
+Dannenberg
+Dalby
+Currence
+Culwell
+Cuesta
+Croston
+Crossno
+Cromley
+Crisci
+Craw
+Coryell
+Cooter
+Condra
+Columbia
+Colpitts
+Colas
+Coach
+Clink
+Clevinger
+Clermont
+Cistrunk
+Cirilo
+Chirico
+Chiarello
+Cephus
+Cecena
+Cavaliere
+Caughey
+Casimir
+Carwell
+Carlon
+Carbonaro
+Caraveo
+Cantley
+Callejas
+Cagney
+Cadieux
+Cabaniss
+Bushard
+Burlew
+Buras
+Budzinski
+Bucklew
+Bruneau
+Brummer
+Brueggemann
+Brotzman
+Bross
+Broad
+Brittian
+Brimage
+Briles
+Brickman
+Breneman
+Breitenstein
+Brandel
+Brackins
+Boydstun
+Botta
+Bosket
+Boros
+Borgmann
+Bordeau
+Bonifacio
+Bolten
+Boehman
+Blundell
+Bloodsaw
+Bjerke
+Biffle
+Bickett
+Bickers
+Beville
+Bergren
+Bergey
+Benzing
+Belfiore
+Beirne
+Beckert
+Bebout
+Baumert
+Battey
+Bartman
+Barrs
+Barriere
+Barcelo
+Barbe
+Balliet
+Baham
+Babst
+Auton
+Asper
+Asbell
+Arzate
+Argento
+Arel
+Araki
+Arai
+Apo
+Antley
+Amodeo
+Ammann
+Allyn
+Allensworth
+Aldape
+Akey
+Abeita
+Zweifel
+Zeng
+Zeiler
+Zamor
+Zalenski
+Yzaguirre
+Yousef
+Yetman
+Yau
+Wyer
+Woolwine
+Wohlgemuth
+Wohlers
+Wittenberg
+Wingrove
+Wind
+Wimsatt
+Willimas
+Wilkenson
+Wildey
+Wilderman
+Wilczynski
+Wigton
+Whorley
+Wellons
+Welles
+Welle
+Weirich
+Weideman
+Weide
+Weekly
+Weast
+Wasmund
+Warshaw
+Walson
+Waldner
+Walch
+Walberg
+Wagener
+Wageman
+Vrieze
+Vossen
+Vorce
+Voorhis
+Vonderheide
+Viruet
+Vicari
+Verne
+Velasques
+Vautour
+Vartanian
+Varona
+Vankeuren
+Vandine
+Vandermeer
+Ursery
+Underdown
+Uhrich
+Uhlman
+Tworek
+Twine
+Twellman
+Tweedie
+Tutino
+Turmelle
+Tubb
+Troop
+Trivedi
+Triano
+Trevathan
+Treese
+Treanor
+Treacy
+Traina
+Topham
+Toenjes
+Tippetts
+Tieu
+Thomure
+Thatch
+Than
+Tetzlaff
+Tetterton
+Tena
+Tell
+Teamer
+Tappan
+Tank
+Talcott
+Tagg
+Szczepanski
+Syring
+Surace
+Sulzer
+Sugrue
+Sugarman
+Suess
+Styons
+Stwart
+Stupka
+Strey
+Straube
+Strate
+Stoddart
+Stockbridge
+Stjames
+Stinger
+Steimle
+Steenberg
+Start
+Stamand
+Staller
+Stahly
+Stager
+Spurgin
+Sprow
+Sponsler
+Speas
+Spainhour
+Sones
+Smits
+Smelcer
+Slovak
+Slaten
+Singleterry
+Simien
+Sidebottom
+Sibrian
+Shellhammer
+Shelburne
+Shambo
+Sepeda
+Seigel
+Scogin
+Scianna
+Schmoll
+Schmelzer
+Scheu
+Schachter
+Savant
+Sauseda
+Satcher
+Sandor
+Sampsell
+Rugh
+Rufener
+Rudolf
+Rotenberry
+Rossow
+Rossbach
+Roots
+Rollman
+Rodrique
+Rodreguez
+Rodkey
+Roda
+Rising
+Rini
+Riggan
+Rients
+Riedl
+Rhines
+Ress
+Reinbold
+Raschke
+Rardin
+Rain
+Racicot
+Quillin
+Pushard
+Primrose
+Pries
+Pressey
+Precourt
+Pratts
+Postel
+Poppell
+Plumer
+Pingree
+Pieroni
+Pflug
+Petre
+Petrarca
+Peterka
+Peru
+Perkin
+Pergande
+Peranio
+Penna
+Pekar
+Pea
+Paulhus
+Pasquariello
+Parras
+Parmentier
+Para
+Panzer
+Pamplin
+Oviatt
+Osterhoudt
+Ostendorf
+Osmun
+Ortman
+Orloff
+Orban
+Onofrio
+Olveda
+Oltman
+Okeeffe
+Ocana
+Nunemaker
+Novy
+Noffsinger
+Nish
+Niday
+Nethery
+Nestle
+Nemitz
+Neidert
+Nadal
+Nack
+Muszynski
+Munsterman
+Mulherin
+Mortimore
+Morter
+Montesino
+Montalvan
+Montalbano
+Momon
+Moman
+Mom
+Mogan
+Minns
+Millward
+Milling
+Michelsen
+Micheal
+Mewborn
+Metro
+Metayer
+Mensch
+Meloy
+Meggs
+Meaders
+Mcsorley
+Mcmenamin
+Mclead
+Mclauchlin
+Mcguffey
+Mcguckin
+Mcglaughlin
+Mcferron
+Mcentyre
+Mccrum
+Mccawley
+Mcbain
+Mayhue
+Mau
+Matzen
+Matton
+Marsee
+Marrin
+Marland
+Markum
+Mantilla
+Manfre
+Malta
+Makuch
+Madlock
+Maclaren
+Macauley
+Luzier
+Luthy
+Lufkin
+Lucena
+Loudin
+Lothrop
+Lorch
+Lona
+Loll
+Loadholt
+Lisa
+Lippold
+Likes
+Lichtman
+Liberto
+Liakos
+Lewicki
+Levett
+Level
+Lentine
+Leja
+Legree
+Lawhead
+Lauro
+Lauder
+Lard
+Lanman
+Lank
+Laning
+Lama
+Lalor
+Krob
+Kriger
+Kriegel
+Krejci
+Kreisel
+Kozel
+Kos
+Konkel
+Kolstad
+Koenen
+Kocsis
+Knoblock
+Knebel
+Klopfer
+Klee
+Kilday
+Kesten
+Kerbs
+Kempker
+Keathley
+Kazee
+Kawasaki
+Kaur
+Kamer
+Kamaka
+Kallenbach
+Kafka
+Jerrell
+Jehle
+Jaycox
+Jardin
+Jahns
+Ivester
+Hyppolite
+Hyche
+Husbands
+Hur
+Huppert
+Hulin
+Hubley
+Horsey
+Hornak
+Holzwarth
+Holmon
+Hollabaugh
+Holaway
+Hodes
+Hoak
+Hinesley
+Hillwig
+Hillebrand
+Highfield
+Heslop
+Herrada
+Hendryx
+Hellums
+Heit
+Heishman
+Heindel
+Hayslip
+Hayford
+Hastie
+Hartgrove
+Hanus
+Hakim
+Hains
+Hadnott
+Gundersen
+Gulino
+Guidroz
+Guebert
+Gressett
+Greenhouse
+Graydon
+Gramling
+Grahn
+Goupil
+Gory
+Gorelick
+Goodreau
+Goodnough
+Golay
+Going
+Goers
+Glatz
+Gillikin
+Gieseke
+Giammarino
+Getman
+Geronimo
+Gerardo
+Gensler
+Gazda
+Garibaldi
+Gahan
+Fury
+Funderburke
+Fukuda
+Fugitt
+Fuerst
+Fortman
+Forsgren
+Formica
+Fluke
+Flink
+Fitton
+Feltz
+Fekete
+Feit
+Fehrenbach
+Farone
+Farinas
+Faries
+Fagen
+Ewin
+Esquilin
+Esch
+Enderle
+Ellery
+Ellers
+Ekberg
+Egli
+Effinger
+Dymond
+Dulle
+Dula
+Duhe
+Dudney
+Duane
+Dowless
+Dower
+Dorminey
+Dopp
+Dooling
+Domer
+Disher
+Dillenbeck
+Difilippo
+Dibernardo
+Deyoe
+Devillier
+Denley
+Deland
+Defibaugh
+Deeb
+Debow
+Dauer
+Datta
+Darcangelo
+Daoust
+Damelio
+Dahm
+Dahlman
+Cypher
+Curling
+Curlin
+Cupit
+Culton
+Cuenca
+Cropp
+Croke
+Cremer
+Crace
+Cosio
+Corzine
+Coombe
+Coman
+Colone
+Coloma
+Collingwood
+Coletta
+Coderre
+Cocke
+Cobler
+Claybrook
+Circle
+Cincotta
+Cimmino
+Christoff
+Christina
+Chisum
+Chillemi
+Chevere
+Chae
+Chachere
+Cervone
+Cermak
+Cefalu
+Cauble
+Cather
+Caso
+Carns
+Carcamo
+Carbo
+Capoccia
+Capello
+Capell
+Canino
+Cambareri
+Calvi
+Cabiness
+Bushell
+Burtt
+Burstein
+Burkle
+Bunner
+Bundren
+Buechler
+Bryand
+Bruso
+Brownstein
+Brow
+Brouse
+Brodt
+Broaden
+Brisbin
+Brightman
+Bridgett
+Brenes
+Breitenbach
+Brazzell
+Brazee
+Bramwell
+Bramhall
+Bradstreet
+Boyton
+Bowland
+Boulter
+Bossert
+Bonura
+Bonebrake
+Bonacci
+Boeck
+Blystone
+Birchard
+Bilal
+Biddy
+Bibee
+Bevans
+Bethke
+Bertelsen
+Berney
+Bergfeld
+Benware
+Bellon
+Bellah
+Been
+Batterton
+Barberio
+Bamber
+Bagdon
+Badeaux
+Averitt
+Augsburger
+Ates
+Arvie
+Aronowitz
+Arens
+Arch
+Araya
+Angelos
+Andrada
+Amell
+Amante
+Alvin
+Almy
+Almquist
+Alls
+Aispuro
+Aguillon
+Agudelo
+Admire
+Acy
+Aceto
+Abbot
+Abalos
+Zdenek
+Zaremba
+Zaccaria
+Youssef
+Wrona
+Wrinkle
+Wrede
+Wotton
+Woolston
+Wolpert
+Wollman
+Wince
+Wimberley
+Willmore
+Willetts
+Wikoff
+Wieder
+Wickert
+Whitenack
+Wernick
+Welte
+Welden
+Weiskopf
+Weisenberger
+Weich
+Wallington
+Walder
+Vossler
+Vore
+Vigo
+Vierling
+Victorine
+Verdun
+Vencill
+Vena
+Vazguez
+Vassel
+Vanzile
+Vanvliet
+Vantrease
+Vannostrand
+Vanderveer
+Vanderveen
+Vancil
+Uyeda
+Umphrey
+Uhler
+Uber
+Tutson
+Turrentine
+Tullier
+Tugwell
+Trundy
+Tripodi
+Tomer
+Tomei
+Tomasi
+Tomaselli
+Tokarski
+Tisher
+Tibbets
+Thweatt
+Thistle
+Tharrington
+Tesar
+Telesco
+Teasdale
+Tatem
+Taniguchi
+Suriel
+Sudler
+Stutsman
+Sturman
+Strite
+Strelow
+Streight
+Strawder
+Stransky
+Strahl
+Stours
+Stong
+Stinebaugh
+Stilts
+Stillson
+Steyer
+Stelle
+Steffy
+Steffensmeier
+Statham
+Squillante
+Spiess
+Spargo
+Southward
+Soller
+Soden
+Snuggs
+Snellgrove
+Smyers
+Smiddy
+Slonaker
+Skyles
+Skowron
+Sivils
+Siqueiros
+Siers
+Siddall
+Shorty
+Shontz
+Shingler
+Shiley
+Shibley
+Sherard
+Shelnutt
+Shedrick
+Shasteen
+Sereno
+Selke
+Scovil
+Scola
+Schuett
+Schuessler
+Schreckengost
+Schranz
+Schoepp
+Schneiderman
+Schlanger
+Schiele
+Scheuermann
+Schertz
+Scheidler
+Scheff
+Schaner
+Schamber
+Scardina
+Savedra
+Saulnier
+Sater
+Sarro
+Sambrano
+Salomone
+Sabourin
+Ruud
+Rutten
+Ruffino
+Ruddock
+Rowser
+Roussell
+Rosengarten
+Rominger
+Rollinson
+Rohman
+Roeser
+Rodenberg
+Roberds
+Ridgell
+Rhodus
+Reynaga
+Rexrode
+Revelle
+Rempel
+Remigio
+Reising
+Reiling
+Reetz
+Rayos
+Ravenscroft
+Ravenell
+Raulerson
+Rasmusson
+Rask
+Rase
+Ragon
+Quesnel
+Quashie
+Puzo
+Puterbaugh
+Ptak
+Prost
+Prisbrey
+Principe
+Pricer
+Pratte
+Pouncey
+Portman
+Pontious
+Pomerantz
+Platter
+Planck
+Pilkenton
+Pilarski
+Piano
+Phegley
+Pertuit
+Perla
+Penta
+Pelc
+Peffer
+Pech
+Peagler
+Pavelka
+Pavao
+Patman
+Paskett
+Parrilla
+Pardini
+Papazian
+Panter
+Palin
+Paley
+Pai
+Pages
+Paetzold
+Packett
+Pacheo
+Ostrem
+Orsborn
+Olmedo
+Okamura
+Oiler
+Ohm
+Oglesbee
+Oatis
+Oakland
+Nuckles
+Notter
+Nordyke
+Nogueira
+Niswander
+Nibert
+Nesby
+Neloms
+Nading
+Naab
+Munns
+Mullarkey
+Moudy
+Moret
+Monnin
+Molder
+Modisette
+Moczygemba
+Moctezuma
+Mischke
+Miro
+Mings
+Milot
+Milledge
+Milhorn
+Milera
+Mieles
+Mickley
+Michelle
+Micek
+Metellus
+Mersch
+Merola
+Mercure
+Mencer
+Mellin
+Mell
+Meinke
+Mcquillan
+Mcmurtrie
+Mckillop
+Mckiernan
+Mckendrick
+Mckamie
+Mcilvaine
+Mcguffie
+Mcgonigle
+Mcgarrah
+Mcfetridge
+Mcenaney
+Mcdow
+Mccutchan
+Mccallie
+Mcadam
+Maycock
+Maybee
+Mattei
+Massi
+Masser
+Masiello
+Marth
+Marshell
+Marmo
+Marksberry
+Markell
+Marchal
+Manross
+Manganaro
+Mally
+Mallow
+Mailhot
+Magyar
+Madonna
+Madero
+Madding
+Maddalena
+Macfarland
+Lynes
+Lush
+Lugar
+Luckie
+Lucca
+Lovitt
+Loveridge
+Loux
+Loth
+Loso
+Lorenzana
+Lorance
+Lockley
+Lockamy
+Littler
+Litman
+Litke
+Liebel
+Lichtenberger
+Licea
+Leverich
+Letarte
+Lesesne
+Leno
+Legleiter
+Leffew
+Laurin
+Launius
+Laswell
+Lassen
+Lasala
+Laraway
+Laramore
+Landrith
+Lancon
+Lanahan
+Laiche
+Laford
+Lachermeier
+Kunst
+Kugel
+Kuck
+Kuchta
+Kube
+Korus
+Koppes
+Kolbe
+Koerber
+Kochan
+Knittel
+Kluck
+Kleve
+Kleine
+Kitch
+Kirton
+Kirker
+Kintz
+Kinghorn
+Kindell
+Kimrey
+Kilduff
+Kilcrease
+Kicklighter
+Kibble
+Kervin
+Keplinger
+Keogh
+Kellog
+Keeth
+Kealey
+Kazmierczak
+Karner
+Kamel
+Kalina
+Kaczynski
+Juel
+Joye
+Jerman
+Jeppson
+Jawad
+Jasik
+Jaqua
+Janusz
+Janco
+Island
+Inskeep
+Inks
+Ingold
+Ing
+Hyndman
+Hymer
+Hunte
+Hunkins
+Humber
+Huffstutler
+Huffines
+Hudon
+Hudec
+Hovland
+Houze
+Hout
+Hougland
+Hopf
+Hon
+Holsapple
+Holness
+Hollenbach
+Hoffmeister
+Hitchings
+Hirata
+Hieber
+Hickel
+Hewey
+Herriman
+Hermansen
+Herandez
+Henze
+Heffelfinger
+Hedgecock
+Hazlitt
+Hazelrigg
+Haycock
+Harren
+Harnage
+Harling
+Harcrow
+Hannold
+Hanline
+Hanel
+Hanberry
+Hammersley
+Hamernik
+Halliwell
+Hajduk
+Haithcock
+Haff
+Hadaway
+Haan
+Gullatt
+Guilbault
+Guidotti
+Gruner
+Grisson
+Grieves
+Granato
+Gracie
+Grabert
+Gover
+Gorka
+Glueck
+Girardin
+Giorgio
+Giesler
+Gersten
+Gering
+Geers
+Gaut
+Gaulin
+Gaskamp
+Garbett
+Gallivan
+Galland
+Gaeth
+Fullenkamp
+Fullam
+Friedrichs
+Freire
+Freeney
+Fredenburg
+Frappier
+Fowkes
+Foree
+Fleurant
+Fleig
+Fleagle
+Fitzsimons
+Fischetti
+Fiorenza
+Finneran
+Filippi
+Figueras
+Fesler
+Fertig
+Fennel
+Feltmann
+Felps
+Felmlee
+Faye
+Fannon
+Familia
+Fairall
+Fail
+Fadden
+Esslinger
+Enfinger
+Elsasser
+Elmendorf
+Ellisor
+Einhorn
+Ehrman
+Egner
+Edmisten
+Edlund
+Ebinger
+Dyment
+Dykeman
+Durling
+Dunstan
+Dunsmore
+Dugal
+Duer
+Drescher
+Doyel
+Down
+Dossey
+Donelan
+Dockstader
+Dobyns
+Divis
+Dilks
+Didier
+Desrosier
+Desanto
+Deppe
+Deng
+Delosh
+Delange
+Defrank
+Debo
+Dauber
+Dartez
+Daquila
+Dankert
+Dahn
+Cygan
+Cusic
+Curfman
+Croghan
+Croff
+Criger
+Creviston
+Crays
+Cravey
+Crandle
+Crail
+Crago
+Craghead
+Cousineau
+Couchman
+Cothron
+Corella
+Conine
+Coller
+Colberg
+Cogley
+Coatney
+Coale
+Clendenin
+Claywell
+Clagon
+Cifaldi
+Choiniere
+Chickering
+Chica
+Chennault
+Chavarin
+Chattin
+Chaloux
+Challis
+Cesario
+Certain
+Cazarez
+Caughman
+Catledge
+Casebolt
+Carrel
+Carra
+Carlow
+Capote
+Canez
+Camillo
+Caliendo
+Calbert
+Cairo
+Bylsma
+Bustle
+Buskey
+Buschman
+Burkhard
+Burghardt
+Burgard
+Buonocore
+Bunkley
+Bungard
+Bundrick
+Bumbrey
+Buice
+Buffkin
+Brundige
+Brockwell
+Brion
+Brin
+Briant
+Bredeson
+Bransford
+Brannock
+Brakefield
+Brackens
+Brabant
+Boxer
+Bowdoin
+Bouyer
+Bothe
+Boor
+Bonavita
+Bollig
+Blurton
+Blunk
+Blanke
+Blanck
+Birden
+Bierbaum
+Bevington
+Beutler
+Betters
+Bettcher
+Bera
+Benway
+Bengston
+Benesh
+Behar
+Bedsole
+Becenti
+Beachy
+Battersby
+Basta
+Bartmess
+Bartle
+Bartkowiak
+Barsky
+Barrio
+Barletta
+Barfoot
+Banegas
+Ballin
+Baldonado
+Bal
+Azcona
+Avants
+Austell
+Aungst
+Aune
+Aumann
+Audia
+Atterbury
+Asselin
+Asmussen
+Ashline
+Asbill
+Arvizo
+Arnot
+Ariola
+Ardrey
+Angstadt
+Anastasio
+Amsden
+Amor
+Amerman
+Alred
+Almeda
+Allington
+Alewine
+Alcina
+Alberico
+Alas
+Ahlgren
+Aguas
+Agrawal
+Agosta
+Adolphsen
+Addie
+Acre
+Acey
+Aburto
+Abler
+Zwiebel
+Zuk
+Zepp
+Zentz
+Ybarbo
+Yarberry
+Yamauchi
+Yamashiro
+Wurtz
+Wronski
+Worster
+Wootten
+Wool
+Wongus
+Woltz
+Wolanski
+Witzke
+Withey
+Wisecarver
+Wingham
+Wineinger
+Winegarden
+Windholz
+Wilgus
+Wiesen
+Wieck
+Widrick
+Wickliffe
+Whittenberg
+Westby
+Werley
+Wengert
+Wendorf
+Weimar
+Weick
+Weckerly
+Watrous
+Wasden
+Walford
+Wainright
+Wahlstrom
+Wadlow
+Vrba
+Voisin
+Vives
+Vivas
+Vitello
+Villescas
+Villavicencio
+Villanova
+Vialpando
+Vetrano
+Verona
+Vensel
+Vassell
+Varano
+Vanriper
+Vankleeck
+Vanduyne
+Vanderpol
+Vanantwerp
+Valenzula
+Udell
+Turnquist
+Tuff
+Trickett
+Tremble
+Tramble
+Tingey
+Ting
+Timbers
+Tietz
+Thon
+Thiem
+Then
+Tercero
+Tenner
+Tenaglia
+Teaster
+Tarlton
+Taitt
+Taggert
+Tabon
+Sward
+Swaby
+Suydam
+Surita
+Suman
+Sugar
+Suddeth
+Stumbo
+Studivant
+Strobl
+Stretch
+Streich
+Stow
+Stoodley
+Stoecker
+Stillwagon
+Stickle
+Stellmacher
+Stefanik
+Steedley
+Starbird
+Stake
+Stainback
+Stacker
+Speir
+Spath
+Sommerfeld
+Soltani
+Solie
+Sojka
+Sobota
+Sobieski
+Sobczak
+Smullen
+Sleeth
+Slaymaker
+Skolnick
+Skoglund
+Sires
+Singler
+Silliman
+Shrock
+Shott
+Shirah
+Shimek
+Shepperd
+Sheffler
+Sheeler
+Sharrock
+Sharman
+Shalash
+Seyfried
+Seybold
+Selander
+Seip
+Seifried
+Sedor
+Sedlock
+Sebesta
+Seago
+Scutt
+Scrivens
+Sciacca
+Schultze
+Schoemaker
+Schleifer
+Schlagel
+Schlachter
+Schempp
+Scheider
+Scarboro
+Santi
+Sang
+Sandhu
+Sally
+Salim
+Saia
+Rylander
+Ryburn
+Rutigliano
+Ruocco
+Ruland
+Rudloff
+Rott
+Rosenburg
+Rosenbeck
+Romberger
+Romanelli
+Rohloff
+Rohlfing
+Rodda
+Rodd
+Ritacco
+Rielly
+Rieck
+Rickles
+Rickenbacker
+Rhett
+Respass
+Reisner
+Reineck
+Reighard
+Rehbein
+Rega
+Redwood
+Reddix
+Razor
+Rawles
+Raver
+Rattler
+Ratledge
+Rathman
+Ramsburg
+Raisor
+Radovich
+Radigan
+Quail
+Puskar
+Purtee
+Priestly
+Prestidge
+Presti
+Pressly
+Pozo
+Pottinger
+Portier
+Porta
+Porcelli
+Poplawski
+Polin
+Points
+Poeppelman
+Pocock
+Plump
+Plantz
+Placek
+Piro
+Pinnell
+Pinkowski
+Pietz
+Picone
+Philbeck
+Pflum
+Peveto
+Perret
+Pentz
+Payer
+Paulette
+Patlan
+Paterno
+Papageorge
+Pae
+Overmyer
+Overland
+Osier
+Orwig
+Orum
+Orosz
+Oquin
+Opie
+Oda
+Ochsner
+Oathout
+Nygard
+Norville
+Northway
+Niver
+Nicolson
+Newhart
+Nery
+Neitzel
+Nath
+Nanez
+Mustard
+Murnane
+Mortellaro
+Morreale
+Morino
+Moriarity
+Morgado
+Moorehouse
+Mongiello
+Molton
+Mirza
+Minnix
+Millspaugh
+Milby
+Miland
+Miguez
+Mickles
+Michaux
+Mento
+Melugin
+Melrose
+Melito
+Meinecke
+Mehr
+Meares
+Mcneece
+Mckane
+Mcglasson
+Mcgirt
+Mcgilvery
+Mcculler
+Mccowen
+Mccook
+Mcclintic
+Mccallon
+Mazzotta
+Maza
+Mayse
+Mayeda
+Matousek
+Matley
+Martyn
+Maroon
+Marney
+Marnell
+Marling
+Marcelino
+Manuelito
+Maltos
+Malson
+Maire
+Mahi
+Maffucci
+Macken
+Maass
+Lyttle
+Lynd
+Lyden
+Lukasiewicz
+Luebbers
+Lovering
+Loveall
+Lords
+Longtin
+Lok
+Lobue
+Loberg
+Loan
+Lipka
+Lion
+Linen
+Lightbody
+Lichty
+Levert
+Lev
+Lettieri
+Letsinger
+Lepak
+Lemmond
+Lembke
+Leitz
+Lasso
+Lasiter
+Lango
+Landsman
+Lamirande
+Lamey
+Laber
+Kuta
+Kulesza
+Kua
+Krenz
+Kreiner
+Krein
+Kreiger
+Kraushaar
+Kottke
+Koser
+Kornreich
+Kopczynski
+Konecny
+Kok
+Koff
+Koehl
+Kocian
+Knaub
+Kmetz
+Kluender
+Klenke
+Kleeman
+Kitzmiller
+Kirsh
+Kilman
+Kildow
+Kielbasa
+Ketelsen
+Kesinger
+Kendra
+Kehr
+Keef
+Kauzlarich
+Karter
+Kahre
+Junk
+Jong
+Jobin
+Joaquin
+Jinkins
+Jines
+Jeffress
+Jaquith
+Jaillet
+Jablonowski
+Ishikawa
+Irey
+Ingerson
+Indelicato
+In
+Huntzinger
+Huisman
+Huett
+Howson
+Houge
+Hosack
+Hora
+Hoobler
+Holtzen
+Holtsclaw
+Hollingworth
+Hollin
+Hoberg
+Hobaugh
+Hilker
+Hilgefort
+Higgenbotham
+Heyen
+Hetzler
+Hessel
+Hennessee
+Hendrie
+Hellmann
+Heft
+Heesch
+Haymond
+Haymon
+Haye
+Havlik
+Havis
+Haverland
+Haus
+Harstad
+Harriston
+Harm
+Harju
+Hardegree
+Hankey
+Hands
+Hampshire
+Hammell
+Hamaker
+Halbrook
+Halberg
+Guptill
+Guntrum
+Gunderman
+Gunder
+Gularte
+Guarnieri
+Gu
+Groll
+Grippo
+Greely
+Grave
+Gramlich
+Goh
+Goewey
+Goetzinger
+Goding
+Giraud
+Giefer
+Giberson
+Gennaro
+Gemmell
+Gearing
+Gayles
+Gaudin
+Gatz
+Gatts
+Gasca
+Garn
+Gandee
+Gammel
+Galindez
+Galati
+Gagliardo
+Fulop
+Fukushima
+Friedt
+Fretz
+Frenz
+Freeberg
+Frederic
+Fravel
+Fountaine
+Forry
+Forck
+Fonner
+Flippin
+Flewelling
+Flansburg
+Filippone
+Fettig
+Fenlon
+Felter
+Felkins
+Fein
+Faz
+Favor
+Favero
+Faulcon
+Farver
+Farless
+Fahnestock
+Facemire
+Faas
+Eyer
+Evett
+Every
+Esses
+Escareno
+Ensey
+Ennals
+Engelking
+Empey
+Emily
+Elvira
+Ellithorpe
+Effler
+Edling
+Edgley
+Durrell
+Dunkerson
+Draheim
+Domina
+Dombrosky
+Doescher
+Dobbin
+Divens
+Dinatale
+Dimitri
+Dieguez
+Diede
+Devivo
+Devilbiss
+Devaul
+Determan
+Desjardin
+Deshaies
+Demo
+Delpozo
+Delorey
+Delman
+Delapp
+Delamater
+Deibert
+Degroff
+Debelak
+Dapolito
+Dano
+Dacruz
+Dacanay
+Cushenberry
+Cruze
+Crosbie
+Cregan
+Cousino
+Corrie
+Corrao
+Corney
+Cookingham
+Conry
+Collingsworth
+Coldren
+Cobian
+Coate
+Clauss
+Chrysler
+Christine
+Christenberry
+Chmiel
+Chauez
+Charters
+Chait
+Cesare
+Cella
+Caya
+Castenada
+Cashen
+Captain
+Cantrelle
+Canova
+Candy
+Canary
+Campione
+Camel
+Calixte
+Caicedo
+Byerley
+Buttery
+Butter
+Burda
+Burchill
+Bun
+Bulmer
+Bulman
+Buesing
+Buczek
+Buckholz
+Buchner
+Buchler
+Buban
+Bryne
+Brutus
+Brunkhorst
+Brumsey
+Brumer
+Brownson
+Broker
+Brodnax
+Brezinski
+Brazile
+Braverman
+Brasil
+Branning
+Bradly
+Boye
+Boulden
+Bough
+Bossard
+Bosak
+Borth
+Borgmeyer
+Borge
+Blowers
+Blaschke
+Blann
+Blankenbaker
+Bisceglia
+Billingslea
+Bialek
+Beverlin
+Besecker
+Berquist
+Benigno
+Benavente
+Belizaire
+Beisner
+Behrman
+Beausoleil
+Bea
+Baylon
+Bayley
+Bassi
+Basnett
+Basilio
+Basden
+Basco
+Banerjee
+Balli
+Bake
+Bagnell
+Bady
+Averette
+Augusta
+Arzu
+Arn
+Archambeault
+Arboleda
+Arbaugh
+Arata
+Antrim
+Amrhein
+Amerine
+Alpers
+Alfrey
+Alcon
+Albus
+Albertini
+Aguiniga
+Aday
+Acquaviva
+Accardi
+Zygmont
+Zych
+Zollner
+Zobel
+Zinck
+Zertuche
+Zaragosa
+Zale
+Zaldivar
+Ying
+Yeadon
+Wykoff
+Woullard
+Wolfrum
+Wohlford
+Wison
+Wiseley
+Wisecup
+Winchenbach
+Wiltsie
+Whittlesey
+Whitelow
+Whiteford
+Wever
+Westrich
+Wertman
+Wensel
+Wenrich
+Weisbrod
+Weglarz
+Wedderburn
+Weatherhead
+Wease
+Warring
+Wand
+Wadleigh
+Voltz
+Vise
+Villano
+Vicario
+Vermeulen
+Vazques
+Vasko
+Varughese
+Vangieson
+Vanfossen
+Vanepps
+Vanderploeg
+Vancleve
+Valerius
+Uyehara
+Unsworth
+Twersky
+Turrell
+Tuner
+Tsui
+Trunzo
+Trousdale
+Trentham
+Traughber
+Torgrimson
+Toppin
+Tokar
+Tobia
+Tippens
+Tigue
+Thong
+Thiry
+Thackston
+Terhaar
+Tenny
+Tassin
+Tadeo
+Sweigart
+Sutherlin
+Sumrell
+Suen
+Stuhr
+Strzelecki
+Strosnider
+Streiff
+Stottlemyer
+Storment
+Storlie
+Stonesifer
+Stogsdill
+Stenzel
+Stemen
+Stellhorn
+Steidl
+Stecklein
+Statton
+Staple
+Stangle
+Spratling
+Spoor
+Spight
+Spelman
+Spece
+Spanos
+Spadoni
+Southers
+Sola
+Sobol
+Smyre
+Slaybaugh
+Sizelove
+Sirmons
+Simington
+Silversmith
+Siguenza
+Sieren
+Shelman
+Shawn
+Sharples
+Sharif
+Shack
+Seville
+Sessler
+Serrata
+Serino
+Serafini
+Semien
+Selvey
+Seedorf
+Seckman
+Seawood
+Screws
+Screen
+Scoby
+Scicchitano
+Schorn
+Schommer
+Schnitzer
+Schleusner
+Schlabach
+Schiel
+Schepers
+Schaber
+Scally
+Sautner
+Sartwell
+Santerre
+Sandage
+Salvia
+Salvetti
+Salsman
+Sallis
+Salais
+Saint
+Saeger
+Sable
+Sabat
+Saar
+Ruther
+Russom
+Ruoff
+Rumery
+Rubottom
+Rozelle
+Rowton
+Routon
+Rotolo
+Rostad
+Roseborough
+Rorick
+Ronco
+Rolls
+Roher
+Roberie
+Robare
+Ritts
+Rison
+Rippe
+Rinke
+Ringwood
+Righter
+Rieser
+Rideaux
+Rickerson
+Renfrew
+Releford
+Reinsch
+Reiman
+Reifsteck
+Reidhead
+Redfearn
+Reddout
+Reaux
+Rance
+Ram
+Rado
+Radebaugh
+Quinby
+Quigg
+Provo
+Provenza
+Provence
+Prophet
+Pridgeon
+Praylow
+Powel
+Poulter
+Portner
+Pontbriand
+Police
+Poirrier
+Poirer
+Platero
+Pixler
+Pintor
+Pigman
+Piersall
+Piel
+Pichette
+Phou
+Phillis
+Phillippe
+Pharis
+Phalen
+Petsche
+Perrier
+Penfield
+Pelosi
+Pebley
+Peat
+Pawloski
+Pawlik
+Pavlick
+Pavel
+Patz
+Patout
+Pascucci
+Pasch
+Parrinello
+Parekh
+Pantaleo
+Pannone
+Pankow
+Pangborn
+Pagani
+Pacelli
+Ort
+Orsi
+Oriley
+Orduno
+Oommen
+Olivero
+Okada
+Ocon
+Ocheltree
+Oberman
+Nyland
+Noss
+Norling
+Nolton
+Nobile
+Nitti
+Nishimoto
+Nghiem
+Neuner
+Neuberger
+Neifert
+Negus
+Naval
+Nagler
+Mullally
+Moulden
+Morra
+Morquecho
+Morocco
+Moots
+Monica
+Mizzell
+Mirsky
+Mirabito
+Minardi
+Milholland
+Mikus
+Mijangos
+Michener
+Michalek
+Methvin
+Merrit
+Menter
+Meneely
+Melody
+Meiers
+Mehring
+Mees
+Medal
+Mcwhirt
+Mcwain
+Mcphatter
+Mcnichol
+Mcnaught
+Mclarty
+Mcivor
+Mcginness
+Mcgaughy
+Mcferrin
+Mcfate
+Mcclenny
+Mcclard
+Mccaskey
+Mccallion
+Mcamis
+Mathisen
+Marton
+Marsico
+Mariner
+Marchi
+Mani
+Mangione
+Magda
+Macaraeg
+Lupi
+Lunday
+Lukowski
+Lucious
+Locicero
+Loach
+Littlewood
+Litt
+Litle
+Lipham
+Linley
+Lindon
+Lightford
+Lieser
+Leyendecker
+Lewey
+Lesane
+Lenzi
+Lenart
+Lena
+Leisinger
+Lehrman
+Lefebure
+Leandro
+Lazard
+Laycock
+Laver
+Launer
+Lastrapes
+Lastinger
+Lasker
+Larkey
+Larger
+Lanser
+Lanphere
+Landey
+Lan
+Lampton
+Lamark
+Lager
+Kumm
+Kullman
+Krzeminski
+Krasner
+Kram
+Koran
+Koning
+Kohls
+Kohen
+Kobel
+Kniffen
+Knick
+Kneip
+Knappenberger
+Knack
+Klumpp
+Klausner
+Kitamura
+Kisling
+Kirshner
+Kinloch
+Kingman
+Kin
+Kimery
+Kestler
+Kellen
+Keleher
+Keehn
+Kearley
+Kasprzak
+Kary
+Kampf
+Kamerer
+Kalis
+Kahan
+Kaestner
+Kadel
+Kabel
+Junge
+Juckett
+Joynt
+Jorstad
+Jetter
+Jelley
+Jefferis
+Jeff
+Jeansonne
+Janecek
+Jaffee
+Jacko
+Izzard
+Istre
+Isherwood
+Ipock
+Iannuzzi
+Hypolite
+Hussein
+Humfeld
+Huckleberry
+Hotz
+Hosein
+Honahni
+Holzworth
+Holdridge
+Holdaway
+Holaday
+Hodak
+Hitchman
+Hippler
+Hinchey
+Hillin
+Hiler
+Hibdon
+Hevey
+Heth
+Hepfer
+Henneman
+Hemsley
+Hemmings
+Hemminger
+Helbert
+Helberg
+Heinze
+Heeren
+Hee
+Heber
+Haver
+Hauff
+Haswell
+Harvison
+Hartson
+Harshberger
+Harryman
+Harries
+Hannibal
+Hane
+Hamsher
+Haggett
+Hagemeier
+Haecker
+Haddon
+Haberkorn
+Guttman
+Guttierrez
+Guthmiller
+Guillet
+Guilbert
+Gugino
+Grumbles
+Griffy
+Gregerson
+Greg
+Granada
+Grana
+Goya
+Goranson
+Gonsoulin
+Goettl
+Goertz
+Goe
+Godlewski
+Glandon
+Glad
+Gilsdorf
+Gillogly
+Gilkison
+Giard
+Giampaolo
+Gheen
+Gettings
+Gesell
+Gershon
+Gaumer
+Gartrell
+Garside
+Garrigan
+Garmany
+Garlitz
+Garlington
+Gamet
+Gail
+Fuss
+Furlough
+Funston
+Funaro
+Frix
+Frasca
+Francoeur
+Forshey
+Foose
+Flatley
+Flagler
+Fils
+Fillers
+Fickett
+Feth
+Fennelly
+Fencl
+Felch
+Fedrick
+Febres
+Fazekas
+Farnan
+Fairless
+Ewan
+Etsitty
+Enterline
+Elvin
+Elsworth
+Elliff
+Ell
+Eleby
+Eldreth
+Eidem
+Edgecomb
+Edds
+Ebarb
+Dworkin
+Dusenberry
+Durrance
+Duropan
+Durfey
+Dungy
+Dundon
+Dumbleton
+Duffel
+Dubon
+Dubberly
+Droz
+Drinkwater
+Dressel
+Doughtie
+Doshier
+Dorrell
+Dora
+Dople
+Doonan
+Donadio
+Dollison
+Doig
+Ditzler
+Dishner
+Discher
+Dimaio
+Digman
+Difalco
+Diem
+Devino
+Devens
+Derosia
+Deppen
+Depaola
+Deniz
+Denardo
+Demos
+Demay
+Delgiudice
+Davi
+Danielsen
+Dally
+Dais
+Dahmer
+Cutsforth
+Cusimano
+Curington
+Cumbee
+Cryan
+Crusoe
+Crowden
+Crete
+Cressman
+Crapo
+Cowens
+Coupe
+Councill
+Coty
+Cotnoir
+Correira
+Copen
+Consiglio
+Combes
+Coffer
+Cockrill
+Coad
+Clogston
+Clasen
+Chock
+Chesnutt
+Charrier
+Chain
+Chadburn
+Cerniglia
+Cebula
+Castruita
+Castilla
+Castaldi
+Casebeer
+Casagrande
+Carta
+Carrales
+Carnley
+Cardon
+Carasco
+Capshaw
+Capron
+Cappiello
+Capito
+Canney
+Candela
+Caminiti
+Califano
+Calico
+Calabria
+Caiazzo
+Cahall
+Buscemi
+Burtner
+Burgdorf
+Bureau
+Burdo
+Buffaloe
+Buchwald
+Brwon
+Brunke
+Brummond
+Brumm
+Broe
+Brocious
+Brocato
+Bro
+Britain
+Briski
+Brisker
+Brightwell
+Bresett
+Breiner
+Brazeau
+Braz
+Brayman
+Brandis
+Bramer
+Bradeen
+Boyko
+Bourbon
+Bossi
+Boshart
+Bortle
+Boniello
+Bomgardner
+Bolz
+Bolenbaugh
+Bohling
+Bohland
+Bochenek
+Blust
+Bloxham
+Blowe
+Blish
+Blackwater
+Bjelland
+Biros
+Birkhead
+Biederman
+Bickle
+Bialaszewski
+Bevil
+Beverley
+Beumer
+Bettinger
+Besse
+Bernett
+Bermejo
+Bement
+Belfield
+Beckler
+Beatrice
+Baxendale
+Batdorf
+Bastin
+Bashore
+Bascombe
+Bartlebaugh
+Barsh
+Ballantine
+Bahl
+Badon
+Bachelor
+Autin
+Audie
+Astin
+Askey
+Ascher
+Arrigo
+Arbeiter
+Antes
+Angers
+Amburn
+Amarante
+Alvidrez
+Althaus
+Allmond
+Alfieri
+Aldinger
+Akerley
+Akana
+Aikins
+Ader
+Acebedo
+Accardo
+Abila
+Aberle
+Abele
+Abboud
+Zollars
+Zimmerer
+Zieman
+Zerby
+Zelman
+Zellars
+Yule
+Yoshimura
+Yonts
+Yeats
+Yant
+Yamanaka
+Wyland
+Wuensche
+Worman
+Wordlaw
+Wohl
+Winslett
+Winberg
+Wilmeth
+Willcutt
+Wiers
+Wiemer
+Wickwire
+Wichman
+Whitting
+Whidbee
+Westergard
+Wemmer
+Wellner
+Weishaupt
+Weinert
+Weedon
+Waynick
+Wasielewski
+Waren
+Walworth
+Wallingford
+Walke
+Waechter
+Viviani
+Vitti
+Villagrana
+Vien
+Vicks
+Venema
+Varnes
+Varnadoe
+Varden
+Vanpatten
+Vanorden
+Vanderzee
+Vandenburg
+Vandehey
+Valls
+Vallarta
+Valderrama
+Valade
+Urman
+Ulery
+Tusa
+Tuft
+Tripoli
+Trimpe
+Trickey
+Tortora
+Torrens
+Torchia
+Toft
+Tjaden
+Tison
+Tindel
+Thurmon
+Thode
+Tardugno
+Tancredi
+Taketa
+Taillon
+Tagle
+Sytsma
+Symes
+Swindall
+Swicegood
+Swartout
+Sundstrom
+Sumners
+Sulton
+Studstill
+Student
+Stroop
+Stonerock
+Stmarie
+Stlawrence
+Stemm
+Steinhauser
+Steinert
+Steffensen
+Stefano
+Stefaniak
+Starck
+Stalzer
+Spidle
+Spake
+Sowinski
+Sosnowski
+Sorber
+Somma
+Soliday
+Soldner
+Soja
+Soderstrom
+Soder
+Sockwell
+Sobus
+Snowball
+Sloop
+Skeeter
+Sinner
+Sinkfield
+Simerly
+Silguero
+Sigg
+Siemers
+Siegmund
+Sidle
+Shum
+Sholtis
+Shkreli
+Sheikh
+Shattles
+Sharlow
+Shao
+Shambaugh
+Shaikh
+Serrao
+Serafino
+Selley
+Selle
+Seel
+Sedberry
+Secord
+Seat
+Schunk
+Schuch
+Schor
+Scholze
+Schnee
+Schmieder
+Schleich
+Schimpf
+Scherf
+Satterthwaite
+Sasson
+Sarkisian
+Sarinana
+Sanzone
+Salvas
+Salone
+Salido
+Saiki
+Sahr
+Rusher
+Rusek
+Ruse
+Ruppel
+Rubi
+Rubel
+Rough
+Rothfuss
+Rothenberger
+Rossell
+Rosenquist
+Rosebrook
+Romito
+Romines
+Rolando
+Rolan
+Roker
+Roehrig
+Rockhold
+Rocca
+Robuck
+Riss
+Rinaldo
+Right
+Riggenbach
+Rezentes
+Reuther
+Reuben
+Renolds
+Rench
+Remus
+Remsen
+Reller
+Relf
+Reitzel
+Reiher
+Rehder
+Redeker
+Ramero
+Rahaim
+Radice
+Quijas
+Qualey
+Purgason
+Prum
+Proudfoot
+Prock
+Probert
+Printup
+Primer
+Primavera
+Prenatt
+Pratico
+Polich
+Podkowka
+Podesta
+Plattner
+Plasse
+Plamondon
+Pittmon
+Pippenger
+Pineo
+Pierpont
+Petzold
+Petz
+Pettiway
+Petters
+Petroski
+Petrik
+Pesola
+Pershall
+Perlmutter
+Penepent
+Peevy
+Pechacek
+Pears
+Peaden
+Pazos
+Pavia
+Pascarelli
+Parm
+Parillo
+Parfait
+Paoletti
+Palomba
+Palencia
+Pagaduan
+Oxner
+Overfield
+Overcast
+Oullette
+Ouk
+Ostroff
+Osei
+Omarah
+Olenick
+Olah
+Odem
+Nygren
+Notaro
+Northcott
+Nodine
+Nilges
+Neyman
+Neve
+Neuendorf
+Neptune
+Neisler
+Neault
+Narciso
+Naff
+Muscarella
+Mun
+Most
+Morrisette
+Morphew
+Morein
+Mor
+Montville
+Montufar
+Montesinos
+Monterroso
+Mongold
+Mona
+Mojarro
+Moitoso
+Mode
+Mirarchi
+Mirando
+Minogue
+Milici
+Miga
+Midyett
+Michna
+Mey
+Meuser
+Messana
+Menzie
+Menz
+Mendicino
+Melone
+Mellish
+Meller
+Melle
+Meints
+Mechem
+Mealer
+Mcwilliam
+Mcwhite
+Mcquiggan
+Mcphillips
+Mcpartland
+Mcnellis
+Mcmackin
+Mclaughin
+Mckinny
+Mckeithan
+Mcguirk
+Mcgillivray
+Mcgarr
+Mcgahee
+Mcfaul
+Mcfadin
+Mceuen
+Mccullah
+Mcconico
+Mcclaren
+Mccaul
+Mccalley
+Mccalister
+Mazer
+Mayson
+Mayhan
+Maugeri
+Mauger
+Mattix
+Mattews
+Maslowski
+Masek
+Martir
+Marsch
+Marquess
+Maron
+Markwell
+Markow
+Marinaro
+Marietta
+Marcinek
+Manner
+Mannella
+Mango
+Mallen
+Majeed
+Mahnke
+Mahabir
+Magby
+Magallan
+Madere
+Machnik
+Lybrand
+Luque
+Lundholm
+Lueders
+Lucian
+Lubinski
+Lowy
+Loew
+Lippard
+Linson
+Lindblad
+Lightcap
+Levitsky
+Levens
+Leonardi
+Lenton
+Lengyel
+Leng
+Leitzel
+Leicht
+Leaver
+Laubscher
+Lashua
+Larusso
+Larrimore
+Lanterman
+Lanni
+Lanasa
+Lamoureaux
+Lambros
+Lamborn
+Lamberti
+Lall
+Lagos
+Lafuente
+Laferriere
+Laconte
+Kyger
+Kupiec
+Kunzman
+Kuehne
+Kuder
+Kubat
+Krogh
+Kreidler
+Krawiec
+Krauth
+Kratky
+Kottwitz
+Korb
+Kono
+Kolman
+Kolesar
+Koeppel
+Knapper
+Klingenberg
+Kjos
+Keppel
+Kennan
+Keltz
+Kealoha
+Kasel
+Karney
+Kanne
+Kamrowski
+Kagawa
+Joo
+Johnosn
+Joesph
+Jilek
+Jarvie
+Jarret
+Jansky
+Jacquemin
+Jacox
+Jacome
+Italiano
+Iriarte
+Ingwersen
+Imboden
+Iglesia
+Huyser
+Hurston
+Hursh
+Huntoon
+Hudman
+Hoying
+Horsman
+Horrigan
+Hornbaker
+Horiuchi
+Hopewell
+Hoop
+Hommel
+Homeyer
+Holzinger
+Holmer
+Hollow
+Hipsher
+Hinchman
+Hilts
+Higginbottom
+Hieb
+Heyne
+Hessling
+Hesler
+Hertlein
+Herford
+Heras
+Henricksen
+Hennemann
+Henery
+Hendershott
+Hemstreet
+Heiney
+Heckert
+Heatley
+Hazell
+Hazan
+Hayashida
+Hausler
+Hartsoe
+Harth
+Harriott
+Harriger
+Harpin
+Hardisty
+Hardge
+Hao
+Hannaman
+Hannahs
+Hamp
+Hammersmith
+Hamiton
+Halsell
+Halderman
+Hagge
+Habel
+Gusler
+Gushiken
+Gurr
+Gummer
+Gullick
+Grunden
+Grosch
+Greenburg
+Greb
+Greaver
+Gratz
+Grajales
+Gourlay
+Gotto
+Gorley
+Goodpasture
+Godard
+Glorioso
+Gloor
+Glascock
+Gizzi
+Giroir
+Gibeault
+Gauldin
+Gauer
+Gartin
+Garrels
+Gamber
+Gallogly
+Galley
+Gade
+Fusaro
+Fripp
+Freyer
+Freiberg
+Franzoni
+Fragale
+Foston
+Forti
+Forness
+Folts
+Followell
+Foard
+Flom
+Fling
+Flett
+Fleitas
+Flamm
+Fino
+Finnen
+Finchum
+Filippelli
+Fickel
+Feucht
+Feiler
+Feenstra
+Feagins
+Faver
+Faux
+Faulkenberry
+Farabaugh
+Fandel
+Fallen
+Faler
+Faivre
+Fairey
+Facey
+Exner
+Evensen
+Erion
+Erben
+Epting
+Epping
+Ephraim
+Engberg
+Elsen
+Ellingwood
+Ellen
+Eisenmann
+Eichman
+Ehle
+Edsall
+Eagles
+Durall
+Dupler
+Dunker
+Dumlao
+Duford
+Duffie
+Dudding
+Dries
+Doung
+Dorantes
+Donahoo
+Domenick
+Dollins
+Dobles
+Dipiazza
+Dino
+Dimeo
+Diehm
+Dicicco
+Devin
+Devenport
+Desormeaux
+Derrow
+Depaolo
+Denver
+Denise
+Demas
+Delpriore
+Delosantos
+Dela
+Degreenia
+Degenhardt
+Defrancesco
+Defenbaugh
+Deets
+Debonis
+Deary
+Dazey
+Dargie
+Dambrosia
+Dalal
+Dagen
+Cun
+Cuen
+Crupi
+Crossan
+Crichlow
+Creque
+Coutts
+Counce
+Coram
+Constante
+Connon
+Collelo
+Coit
+Cocklin
+Coblentz
+Cobey
+Coard
+Clutts
+Clingan
+Claw
+Clampitt
+Claeys
+Ciulla
+Cimini
+Ciampa
+Christon
+Choat
+Chiou
+Chenail
+Chavous
+Catto
+Catalfamo
+Casterline
+Cassinelli
+Caspers
+Carroway
+Carlen
+Carithers
+Cappel
+Calo
+Callow
+Calandra
+Cagley
+Cafferty
+Byun
+Byam
+Buttner
+Buth
+Burtenshaw
+Burget
+Burfield
+Buresh
+Bunt
+Bultman
+Bulow
+Buchta
+Buchmann
+Brunett
+Bruemmer
+Brueggeman
+Britto
+Briney
+Brimhall
+Bribiesca
+Bresler
+Brazan
+Brashier
+Brar
+Brandstetter
+Brandi
+Boze
+Boonstra
+Bluitt
+Blomgren
+Blattner
+Blasi
+Bladen
+Bitterman
+Bilby
+Bierce
+Biello
+Bettes
+Bertone
+Berrey
+Bernat
+Berberich
+Benshoof
+Bendickson
+Below
+Bellefeuille
+Bednarski
+Beddingfield
+Beckerman
+Beaston
+Bavaro
+Batalla
+Basye
+Baskins
+Bartolotta
+Bartkowski
+Barranco
+Barkett
+Band
+Banaszak
+Bame
+Bamberger
+Balsley
+Ballas
+Balicki
+Balding
+Bald
+Badura
+Aymond
+Aylor
+Aylesworth
+Axley
+Axelrod
+Aubert
+Armond
+Ariza
+Apicella
+Anstine
+Ankrom
+Angevine
+Anger
+Andreotti
+Andrea
+Alto
+Alspaugh
+Alpaugh
+Almada
+Allinder
+Alexandra
+Alequin
+Alan
+Aguillard
+Agron
+Agena
+Afanador
+Ackerley
+Abrev
+Abdalla
+Aaronson
+Zynda
+Zucco
+Zipp
+Zetina
+Zenz
+Zelinski
+Youngren
+Yochum
+Yearsley
+Yankey
+Woodfork
+Wohlwend
+Woelfel
+Wiste
+Wismer
+Winzer
+Winker
+Wilkison
+Wigger
+Wierenga
+Whipps
+Wheeling
+Westray
+Wesch
+Weld
+Weible
+Wedell
+Weddell
+Wawrzyniak
+Wasko
+Washinton
+Wantz
+Walts
+Wallander
+Wain
+Wahlen
+Wachowiak
+Voshell
+Viteri
+Vire
+Villafuerte
+Vieyra
+Viau
+Vescio
+Verrier
+Verhey
+Vause
+Vandermolen
+Vanderhorst
+Valois
+Valla
+Valcourt
+Vacek
+Uzzle
+Umland
+Um
+Ulman
+Ulland
+Turvey
+Tuley
+Trembath
+Trees
+Trabert
+Towsend
+Totman
+Toews
+Toby
+Tito
+Tisch
+Tisby
+Tipping
+Tierce
+Thivierge
+Tenenbaum
+Teagle
+Tacy
+Tabler
+Szewczyk
+Swearngin
+Suire
+Sturrock
+Stubbe
+Stronach
+Stoute
+Stoudemire
+Stoneberg
+Sterba
+Stejskal
+Steier
+Stehr
+Steckler
+Steckel
+Stearman
+Steakley
+Star
+Stanforth
+Stancill
+Stalls
+Srour
+Sprowl
+Spevak
+Sole
+Sokoloff
+Soderman
+Snover
+Sleeman
+Slaubaugh
+Sitzman
+Simpler
+Simmer
+Simes
+Siegal
+Sidoti
+Sidler
+Sider
+Sidener
+Siddiqi
+Shireman
+Shima
+Sheroan
+Shadduck
+Seyal
+Sentell
+Sennett
+Senko
+Seneca
+Sen
+Seligman
+Seipel
+Seekins
+Seabaugh
+Scouten
+Schweinsberg
+Schwartzberg
+Schurr
+Schult
+Schrick
+Schoening
+Schmitmeyer
+Schlicher
+Schlager
+Schack
+Schaar
+Scavuzzo
+Scarpa
+Sassano
+Santigo
+Sandavol
+San
+Sampsel
+Samms
+Samet
+Salzano
+Salyards
+Salva
+Saidi
+Sabir
+Saam
+Saab
+Runions
+Rundquist
+Rousselle
+Round
+Rotunno
+Roses
+Rosch
+Romney
+Rohner
+Roff
+Rockhill
+Rockefeller
+Rocamora
+Rm
+Ringle
+Riggie
+Ricklefs
+Rexroat
+Reves
+Revel
+Reuss
+Reta
+Repka
+Rentfro
+Reineke
+Recore
+Recalde
+Rease
+Rawling
+Ravencraft
+Ravelo
+Rappa
+Randol
+Ramsier
+Ramerez
+Rahimi
+Rahim
+Radney
+Racey
+Raborn
+Rabalais
+Quebedeaux
+Pujol
+Puchalski
+Prothro
+Proffit
+Prigge
+Prideaux
+Prevo
+Portales
+Porco
+Popovic
+Popek
+Popejoy
+Pompei
+Plumber
+Plude
+Platner
+Plate
+Pizzuto
+Pizer
+Pistone
+Piller
+Pierri
+Piehl
+Pickert
+Piasecki
+Phong
+Philipp
+Peugh
+Pesqueira
+Perrett
+Perfetti
+Percell
+Penhollow
+Pelto
+Pellett
+Pavlak
+Paulo
+Paula
+Patricia
+Pastorius
+Parsell
+Parrales
+Pareja
+Parcell
+Pappan
+Pajak
+Owusu
+Ovitt
+Ory
+Orrick
+Oniell
+Olliff
+Olberding
+Oesterling
+Odwyer
+Ocegueda
+Obey
+Obermiller
+Nylander
+Nulph
+Nottage
+Northam
+Norgard
+Nodal
+Niel
+Nicols
+Newhard
+Nellum
+Neira
+Nazzaro
+Nassif
+Narducci
+Nalbandian
+Nails
+Musil
+Murga
+Muraoka
+Mumper
+Mulroy
+Mountjoy
+Mossey
+Moreton
+Morea
+Montoro
+Montesdeoca
+Montealegre
+Montanye
+Montandon
+Mok
+Moisan
+Mohl
+Modesto
+Modeste
+Mitra
+Mister
+Minson
+Minjarez
+Milbourne
+Michaelsen
+Metheney
+Mestre
+Mescher
+Mervis
+Mennenga
+Melgarejo
+Meisinger
+Meininger
+Mcwaters
+Mckern
+Mckendree
+Mchargue
+Mcglothlen
+Mcgibbon
+Mcgavock
+Mcduffee
+Mcclurkin
+Mccausland
+Mccardell
+Mccambridge
+Mazzoni
+Mayen
+Maxton
+Mawson
+Mauffray
+Mattinson
+Mattila
+Matsunaga
+Mater
+Mascia
+Marse
+Marotz
+Marois
+Markin
+Markee
+Marcinko
+Marcin
+Manville
+Mantyla
+Manser
+Manry
+Manderscheid
+Mallari
+Malia
+Malecha
+Malcomb
+Majerus
+Mailman
+Macinnis
+Mabey
+Lyford
+Luth
+Lupercio
+Luhman
+Luedke
+Lovick
+Lossing
+Loss
+Lorraine
+Lookabaugh
+Longway
+Lone
+Loisel
+Logiudice
+Loffredo
+Locust
+Lobe
+Lobaugh
+Lizaola
+Livers
+Littlepage
+Linnen
+Limmer
+Liebsch
+Liebman
+Leyden
+Levitan
+Levison
+Levier
+Leven
+Levalley
+Lettinga
+Lessley
+Lessig
+Lepine
+Leight
+Leick
+Leggio
+Leffingwell
+Leffert
+Lefevers
+Ledlow
+Leaton
+Leander
+Leaming
+Lazos
+Laviolette
+Lauffer
+Latz
+Lasorsa
+Lasch
+Larin
+Laporta
+Lanter
+Langstaff
+Landi
+Lamica
+Lambson
+Lambe
+Lamarca
+Laman
+Lamagna
+Lajeunesse
+Lafontant
+Lafler
+Labrum
+Laakso
+Kush
+Kuether
+Kuchar
+Kruk
+Kroner
+Kroh
+Kridler
+Kreuzer
+Kovats
+Koprowski
+Kohout
+Knicely
+Knell
+Klutts
+Kindrick
+Kiddy
+Khanna
+Ketcher
+Kerschner
+Kerfien
+Kensey
+Kenley
+Kenan
+Kemplin
+Kellerhouse
+Keesling
+Keep
+Keena
+Keas
+Kaplin
+Kanady
+Kampen
+Jutras
+Jungers
+Julio
+Jeschke
+Jen
+Janowski
+Janas
+Iskra
+Imperato
+Ikerd
+Igoe
+Hyneman
+Hynek
+Husain
+Hurrell
+Hultquist
+Hullett
+Hulen
+Huf
+Huberty
+Hoyte
+Hossain
+Hornstein
+Hori
+Hopton
+Holms
+Hollmann
+Holdman
+Holdeman
+Holben
+Hoffert
+Himel
+Hillsman
+Hillary
+Herdt
+Hellyer
+Hellen
+Heister
+Heimer
+Heidecker
+Hedgpeth
+Hedgepath
+Hebel
+Heatwole
+Hayer
+Hausner
+Haskew
+Haselden
+Hartranft
+Harsch
+Harres
+Harps
+Hardimon
+Halm
+Hallee
+Hallahan
+Hackley
+Hackenberg
+Hachey
+Haapala
+Guynes
+Gunnerson
+Gunby
+Gulotta
+Gudger
+Groman
+Grignon
+Griebel
+Gregori
+Greenan
+Grauer
+Gourd
+Gorin
+Gorgone
+Gooslin
+Goold
+Goltz
+Goldberger
+Gobble
+Glotfelty
+Glassford
+Glance
+Gladwin
+Giuffre
+Gilpatrick
+Germaine
+Gerdts
+Genna
+Geisel
+Gayler
+Gaunce
+Gaulding
+Gateley
+Gassman
+Gash
+Garson
+Garron
+Garand
+Gangestad
+Gallow
+Galbo
+Gabrielli
+Fullington
+Fucci
+Frum
+Frieden
+Friberg
+Frasco
+Francese
+Fowle
+Foucher
+Fothergill
+Foraker
+Fonder
+Foisy
+Fogal
+Flurry
+Flenniken
+Fitzhenry
+Fishbein
+Finton
+Filmore
+Filice
+Feola
+Felberbaum
+Fausnaught
+Fasciano
+Farrah
+Farquharson
+Faires
+Estridge
+Essman
+Enz
+Enriques
+Emmick
+Ekker
+Ekdahl
+Eisman
+Eggleton
+Eddinger
+Eakle
+Eagar
+Durio
+Dunwoody
+Duhaime
+Duenes
+Duden
+Dudas
+Dresher
+Dresel
+Doutt
+Donlan
+Donathan
+Domke
+Dobrowolski
+Dingee
+Dimmitt
+Dimery
+Dilullo
+Deveaux
+Devalle
+Desper
+Desnoyers
+Desautels
+Derouin
+Derbyshire
+Denmon
+Dena
+Demski
+Delucca
+Delpino
+Delmont
+Deller
+Dejulio
+Deibler
+Dehne
+Deharo
+Degner
+Defore
+Deerman
+Decuir
+Deckman
+Deasy
+Dease
+Deaner
+Dawdy
+Daughdrill
+Darrigo
+Darity
+Daniele
+Dalbey
+Dagenhart
+Daffron
+Curro
+Curnutte
+Curatolo
+Cruikshank
+Crosswell
+Croslin
+Croney
+Crofton
+Criado
+Crecelius
+Coscia
+Conniff
+Commodore
+Coltharp
+Colonna
+Collyer
+Collington
+Cobbley
+Coache
+Clonts
+Cloe
+Cliett
+Clemans
+Clara
+Cid
+Christo
+Chrisp
+China
+Chiarini
+Chia
+Cheatam
+Cheadle
+Che
+Chauncey
+Chand
+Chadd
+Cervera
+Cerulli
+Cerezo
+Cedano
+Cayetano
+Cawthorne
+Cavalieri
+Cattaneo
+Caryl
+Cartlidge
+Carrithers
+Carreira
+Carranco
+Cargle
+Candanoza
+Camille
+Camburn
+Calender
+Calderin
+Calcagno
+Cahn
+Cadden
+Byham
+Buttry
+Burry
+Burruel
+Burkitt
+Burgio
+Burgener
+Buescher
+Buckalew
+Brymer
+Brumett
+Brugnoli
+Brugman
+Brosnahan
+Bronder
+Broeckel
+Broderson
+Brisbon
+Brinsfield
+Brinks
+Bresee
+Bregman
+Branner
+Brambila
+Brailsford
+Bouska
+Boster
+Borucki
+Bortner
+Boroughs
+Borgeson
+Bonier
+Bomba
+Bolender
+Boesch
+Boeke
+Bloyd
+Bley
+Binger
+Billing
+Bilbro
+Biery
+Bichrest
+Bezio
+Bevel
+Berrett
+Bermeo
+Bergdoll
+Bercier
+Benzel
+Bentler
+Bennetts
+Belnap
+Bellini
+Beitz
+Behrend
+Bednarczyk
+Bearse
+Batman
+Bartolini
+Bartol
+Barretta
+Barbero
+Barbaro
+Banvelos
+Bankes
+Ballengee
+Baldon
+Aye
+Ausmus
+Atilano
+Atienza
+Aschenbrenner
+Arora
+Armstong
+Aquilino
+Appleberry
+Applebee
+Apolinar
+Antos
+Angles
+Andrepont
+Ancona
+Amesquita
+Alvino
+Altschuler
+Allin
+Alire
+Ainslie
+Agular
+Aeschliman
+Accetta
+Abdulla
+Abbe
+Zwart
+Zufelt
+Zona
+Zirbel
+Zingaro
+Zilnicki
+Zenteno
+Zent
+Zemke
+Zayac
+Zarrella
+Yoshimoto
+Yearout
+Wrench
+World
+Womer
+Woltman
+Wolin
+Wolery
+Woldt
+Witts
+Wittner
+Witherow
+Winward
+Winrow
+Wiemann
+Wichmann
+Whitwell
+Whitelaw
+Wheeless
+Whalley
+Wey
+Wessner
+Wenzl
+Wene
+Weatherbee
+Waye
+Wattles
+Wanke
+Walkes
+Waldeck
+Vonruden
+Voisine
+Vogus
+Vittetoe
+Villalva
+Villacis
+Victorian
+Verge
+Venturini
+Venturi
+Venson
+Vanloan
+Vanhooser
+Vanduzer
+Vandever
+Vanderwal
+Vanderheyden
+Vanbeek
+Vanbebber
+Vallance
+Vales
+Vahle
+Urbain
+Upshur
+Umfleet
+Twist
+Tsuji
+Trybus
+Triolo
+Trimarchi
+Trezza
+Trenholm
+Tovey
+Tourigny
+Torry
+Torrain
+Torgeson
+Tongue
+Tomey
+Tischler
+Tinkler
+Tinder
+Ticknor
+Tibbles
+Tibbals
+Throneberry
+Thormahlen
+Thibert
+Thibeaux
+Theurer
+Templet
+Tegeler
+Tavernier
+Taubman
+Tamashiro
+Tallon
+Tallarico
+Taboada
+Sypher
+Sybert
+Swyers
+Switalski
+Swinger
+Swedberg
+Suther
+Surprenant
+Sullen
+Sulik
+Sugden
+Suder
+Suchan
+Such
+Strube
+Stroope
+Strittmatter
+Streett
+Straughn
+Strasburg
+Stjacques
+Stimage
+Stimac
+Stifter
+Stgelais
+Steinhart
+Stehlik
+Steffenson
+Steenbergen
+Stanbery
+Stallone
+Sprung
+Spraggs
+Spoto
+Spilman
+Speno
+Spanbauer
+Spalla
+Spagnolo
+Soliman
+Solan
+Sobolik
+Snelgrove
+Snedden
+Smale
+Sliter
+Slankard
+Sircy
+Signor
+Shutter
+Shurtliff
+Shur
+Show
+Shirkey
+Shi
+Shewmake
+Shams
+Shadley
+Shaddox
+Sgro
+Serfass
+Seppala
+Segawa
+Segalla
+Seaberry
+Scruton
+Scism
+Schwein
+Schwartzman
+Schwantes
+Schomer
+Schoenborn
+Schlottmann
+Schissler
+Scheurer
+Schepis
+Scheidegger
+Saunier
+Sauders
+Sassman
+Sannicolas
+Sanderfur
+Salser
+Sagar
+Saffer
+Saeed
+Sadberry
+Saban
+Ryce
+Rybak
+Rux
+Rumore
+Rummell
+Rummage
+Rudasill
+Rozman
+Rota
+Rossin
+Rosell
+Rosel
+Romberg
+Rojero
+Rochin
+Rochell
+Robideau
+Robarge
+Roath
+Risko
+Ringel
+Ringdahl
+Riera
+Riemann
+Ribas
+Revard
+Renna
+Renegar
+Reinwald
+Rehman
+Regal
+Reels
+Ree
+Redel
+Reasons
+Raysor
+Rathke
+Rapozo
+Rampton
+Ramaker
+Rakow
+Raia
+Radin
+Raco
+Rackham
+Racca
+Racanelli
+Rabun
+Quaranta
+Purves
+Pundt
+Protsman
+Prosper
+Prezioso
+Presutti
+President
+Presgraves
+Poydras
+Portnoy
+Portalatin
+Pop
+Pontes
+Poehler
+Poblete
+Poat
+Plumadore
+Pleiman
+Pizana
+Piscopo
+Piraino
+Pinelli
+Pillai
+Picken
+Picha
+Piccoli
+Philen
+Petteway
+Petros
+Peskin
+Perugini
+Perrella
+Pernice
+Peper
+Pensinger
+Pembleton
+Patron
+Passman
+Parrent
+Panetta
+Pancake
+Pallas
+Palka
+Pais
+Paglia
+Padmore
+Oum
+Ottesen
+Ost
+Oser
+Ortmann
+Ormand
+Oriol
+Orick
+Oler
+Okafor
+Ohair
+Obert
+Oberholtzer
+Number
+Nowland
+Nosek
+Nordeen
+Nolf
+Nogle
+Nobriga
+Nicley
+Niccum
+Newingham
+Neumeister
+Neugebauer
+Netherland
+Nerney
+Neiss
+Neis
+Neider
+Neeld
+Nailor
+Mustain
+Mussman
+Musante
+Murton
+Murden
+Munyon
+Muldrew
+Motton
+Moscoso
+Moschella
+Moroz
+Mormon
+Morelos
+Morace
+Moone
+Montesano
+Montemurro
+Montas
+Montalbo
+Molander
+Mleczko
+Miyake
+Mitschke
+Minger
+Minelli
+Minear
+Millener
+Mihelich
+Miedema
+Miah
+Metzer
+Mery
+Merrigan
+Merck
+Mennella
+Membreno
+Melecio
+Melder
+Mehling
+Mehler
+Medcalf
+Meche
+Mealing
+Mcqueeney
+Mcphaul
+Mcmickle
+Mcmeen
+Mcmains
+Mclees
+Mcgowin
+Mcfarlain
+Mcdivitt
+Mccotter
+Mcconn
+Mcclane
+Mccaster
+Mcbay
+Mcbath
+Mayoral
+Mayeux
+Matsuo
+Masur
+Massman
+Marzette
+Martensen
+Marlett
+Markie
+Markgraf
+Marcinkowski
+Marchbanks
+Marcella
+Mansir
+Mandez
+Mancil
+Malagon
+Magnani
+Madonia
+Madill
+Madia
+Mackiewicz
+Macgillivray
+Macdowell
+Macbeth
+Mabee
+Lundblad
+Lovvorn
+Lovings
+Loreto
+Linz
+Linwood
+Linnell
+Linebaugh
+Lindstedt
+Lindbloom
+Linda
+Limberg
+Liebig
+Lickteig
+Lichtenberg
+Licari
+Lex
+Lewison
+Levario
+Levar
+Lepper
+Lenzen
+Lenderman
+Lemarr
+Leinen
+Leider
+Legrande
+Lefort
+Lebleu
+Leask
+Learn
+Leacock
+Lazano
+Lawalin
+Laven
+Laplaca
+Lant
+Langsam
+Langone
+Landress
+Landen
+Lande
+Lamorte
+Lairsey
+Laidlaw
+Laffin
+Lackner
+Lacaze
+Labuda
+Labree
+Labella
+Labar
+Kyer
+Kuyper
+Kulinski
+Kulig
+Kuhnert
+Kuchera
+Kubicek
+Kruckeberg
+Kruchten
+Krider
+Kotch
+Kornfeld
+Koren
+Koogler
+Koll
+Kole
+Kohnke
+Kohli
+Kofoed
+Koelling
+Kluth
+Klump
+Klopfenstein
+Klippel
+Klinge
+Klett
+Klemp
+Kleis
+Klann
+Kitzman
+Kinnan
+Kingsberry
+Kind
+Kina
+Kilmon
+Killpack
+Kilbane
+Kijowski
+Kies
+Kierstead
+Kettering
+Kesselman
+Kenton
+Kennington
+Keniston
+Kehrer
+Kearl
+Keala
+Kassa
+Kasahara
+Kantz
+Kalin
+Kaina
+Jupin
+Juntunen
+Juares
+Joynes
+Jovel
+Joos
+Jn
+Jiggetts
+Jervis
+Jerabek
+Jennison
+Jaso
+Janz
+Izatt
+Ishibashi
+Iannotti
+Hymas
+Huneke
+Hulet
+Hougen
+Horvat
+Horstmann
+Hopple
+Holtkamp
+Holsten
+Hohenstein
+Hoefle
+Hoback
+Hiney
+Hiemstra
+Herwig
+Herter
+Herriott
+Hermsen
+Herdman
+Herder
+Herbig
+Hem
+Helper
+Helling
+Helbig
+Heitkamp
+Heinrichs
+Heinecke
+Heileman
+Heffley
+Heavrin
+Heaston
+Haymaker
+Hauenstein
+Hartlage
+Harlin
+Harig
+Hardenbrook
+Hankin
+Hamiter
+Hagens
+Hagel
+Grizzell
+Griest
+Griese
+Grief
+Grennan
+Graden
+Gosse
+Gorder
+Goldin
+Goatley
+Gillespi
+Gilbride
+Giel
+Gianni
+Ghoston
+Getter
+Gershman
+Geisinger
+Gehringer
+Gedeon
+Gebert
+Gaxiola
+Gawronski
+Gau
+Gathright
+Gatchell
+Gargiulo
+Garg
+Galang
+Gadison
+Fyock
+Furniss
+Furby
+Funnell
+Frizell
+Frenkel
+Freeburg
+Frankhouser
+Franchi
+Foulger
+Formby
+Forkey
+Fonte
+Folson
+Follette
+Flicker
+Flavors
+Flavell
+Finegan
+Fill
+Filippini
+Ferencz
+Ference
+Fennessey
+Feggins
+Feehan
+Fazzino
+Fazenbaker
+Fausto
+Faunce
+Farraj
+Farnell
+Farler
+Farabee
+Falkowski
+Facio
+Etzler
+Ethington
+Esterline
+Esper
+Esker
+Erxleben
+Ericsson
+Erick
+Engh
+Emling
+Elridge
+Ellenwood
+Elfrink
+Ekhoff
+Eisert
+Eis
+Eifert
+Eichenlaub
+Egnor
+Eggebrecht
+Edlin
+Edberg
+Eble
+Eber
+Easler
+Duwe
+Dutta
+Dutremble
+Dusseault
+Durney
+Dunworth
+Dumire
+Dukeman
+Dufner
+Duey
+Duble
+Dreese
+Dozal
+Douville
+Dougal
+Doom
+Done
+Diver
+Ditmore
+Distin
+Dimuzio
+Dildine
+Dignan
+Dieterich
+Dieckman
+Didonna
+Dhillon
+Dezern
+Devereux
+Devall
+Detty
+Detamore
+Derksen
+Deremer
+Deras
+Denslow
+Deno
+Denicola
+Denbow
+Demma
+Demille
+Delisa
+Delira
+Delawder
+Delara
+Delahanty
+Dejonge
+Deininger
+Dedios
+Dederick
+Decelles
+Debus
+Debruyn
+Deborde
+Deak
+Dauenhauer
+Darsey
+Daring
+Dansie
+Dalman
+Dakin
+Dagley
+Czaja
+Cybart
+Cutchin
+Currington
+Curbelo
+Croucher
+Crinklaw
+Cremin
+Cratty
+Cranfield
+Crafford
+Cowher
+Cowboy
+Couvillion
+Couturier
+Counter
+Corter
+Coombes
+Contos
+Consolini
+Connaughton
+Conely
+Coltrane
+Collom
+Cockett
+Clepper
+Cleavenger
+Claro
+Clarkin
+Ciriaco
+Ciesla
+Cichon
+Ciancio
+Cianci
+Chynoweth
+Chuang
+Chrzanowski
+Christion
+Cholewa
+Chipley
+Chilcott
+Cheyne
+Cheslock
+Chenevert
+Cheers
+Charlot
+Chagolla
+Chabolla
+Cesena
+Cerutti
+Cava
+Caul
+Cassone
+Cassin
+Cassese
+Casaus
+Casali
+Cartledge
+Carsten
+Cardamone
+Carcia
+Carbonneau
+Carboni
+Carabello
+Capozzoli
+Capella
+Cap
+Cannata
+Campoverde
+Campeau
+Cambre
+Camberos
+Calvery
+Calnan
+Calmes
+Calley
+Callery
+Calise
+Cacciotti
+Cacciatore
+Butterbaugh
+Burgo
+Burgamy
+Burell
+Bunde
+Bumbalough
+Buel
+Buechner
+Buchannon
+Bryon
+Brunn
+Brost
+Broadfoot
+Brittan
+Brevard
+Breda
+Brazel
+Brayboy
+Brasier
+Boyea
+Boxx
+Both
+Boso
+Bosio
+Boruff
+Borda
+Bongiovanni
+Bolerjack
+Boedeker
+Blye
+Blumstein
+Blumenfeld
+Blinn
+Bleakley
+Blatter
+Blan
+Bjornson
+Bisignano
+Billick
+Bieniek
+Bhatti
+Bevacqua
+Betterton
+Berra
+Berenbaum
+Bensinger
+Bennefield
+Belvins
+Belson
+Bellin
+Beighley
+Beecroft
+Beaudreau
+Baynard
+Bautch
+Bausch
+Basch
+Bartleson
+Barthelemy
+Barak
+Balzano
+Balistreri
+Bailer
+Bagnall
+Bagg
+Bae
+Auston
+Augustyn
+Aslinger
+Ashalintubbi
+Artist
+Arjona
+Arebalo
+Arab
+Appelbaum
+Anna
+Angst
+Angert
+Angelucci
+Andry
+Andersson
+Amorim
+Amavisca
+Alward
+Alvelo
+Alvear
+Alumbaugh
+Alsobrook
+Alli
+Allgeier
+Allende
+Aldrete
+Akiyama
+Ahlquist
+Adolphson
+Addario
+Acoff
+Abelson
+Abasta
+Zulauf
+Zirkind
+Zeoli
+Zemlicka
+Zawislak
+Zappia
+Zanella
+Yelvington
+Yeatman
+Yanni
+Wragg
+Wissing
+Wischmeier
+Wirta
+Wiren
+Wilmouth
+Williard
+Willert
+Willaert
+Wildt
+Whelpley
+Westwood
+Weingart
+Weidenbach
+Weidemann
+Weatherman
+Weakland
+Watwood
+Wattley
+Waterson
+Wambach
+Walzer
+Waldow
+Waag
+Vorpahl
+Volkmann
+Vitolo
+Visitacion
+Vincelette
+Vina
+Viggiano
+Vieth
+Vidana
+Vert
+Verna
+Verges
+Verdejo
+Venzon
+Velardi
+Varian
+Vargus
+Vandermeulen
+Vandam
+Vanasse
+Vanaman
+Utzinger
+Uriostegui
+Uplinger
+Twiss
+Tumlinson
+Tschanz
+Trunnell
+Troung
+Troublefield
+Trojacek
+Trial
+Treloar
+Tranmer
+Touchton
+Torsiello
+Torina
+Tootle
+Toki
+Toepfer
+Tippin
+Tippie
+Thronson
+Thomes
+Tezeno
+Texada
+Testani
+Tessmer
+Terrel
+Terra
+Terlizzi
+Tempel
+Temblador
+Tayler
+Tawil
+Tasch
+Tames
+Talor
+Talerico
+Swinderman
+Sweetland
+Swager
+Sulser
+Sullens
+Subia
+Sturgell
+Stumpff
+Stufflebeam
+Stucki
+Strohmeyer
+Strebel
+Straughan
+Strackbein
+Stobaugh
+Stetz
+Stelter
+Steinmann
+Steinfeld
+Stefani
+Stecher
+Stanwood
+Stanislawski
+Stander
+Speziale
+Soppe
+Soni
+Sol
+Sobotka
+Snipe
+Smuin
+Slider
+Slee
+Skerrett
+Sjoberg
+Sittig
+Simonelli
+Simo
+Sima
+Silvio
+Silverio
+Silveria
+Silsby
+Sillman
+Sienkiewicz
+Sick
+Sia
+Shomo
+Shoff
+Shoener
+Shiba
+Sherfey
+Shehane
+Shawl
+Sexson
+Setton
+Sergi
+Selvy
+Seiders
+Seegmiller
+Sebree
+Seabury
+Scroggin
+Sconyers
+Schwalb
+Schurg
+Schulenberg
+Schuld
+Schrage
+Schow
+Schon
+Schnur
+Schneller
+Schmidtke
+Schlatter
+Schieffer
+Schenkel
+Scheeler
+Schauwecker
+Schartz
+Schacherer
+Scafe
+Sayegh
+Savidge
+Saur
+Sarles
+Sarkissian
+Sarkis
+Sarcone
+Sagucio
+Saffell
+Saenger
+Sacher
+Rylee
+Ruvolo
+Ruston
+Ruple
+Rulison
+Ruge
+Ruffo
+Ruehl
+Rueckert
+Rudman
+Rudie
+Rubert
+Rozeboom
+Roysden
+Roylance
+Rothchild
+Rosse
+Rosecrans
+Rodrick
+Rodi
+Rockmore
+Robnett
+Roberti
+Rivett
+Riva
+Ritzel
+Rierson
+Ricotta
+Ricken
+Rezac
+Rendell
+Remo
+Reitman
+Reindl
+Reeb
+Reddic
+Reddell
+Rebuck
+Reali
+Raye
+Raso
+Ramthun
+Ramsden
+Rameau
+Ralphs
+Rak
+Rago
+Racz
+Quinteros
+Quinter
+Quinley
+Quiggle
+Quaid
+Purvines
+Purinton
+Purdum
+Pummill
+Puglia
+Puett
+Ptacek
+Przybyla
+Prowse
+Providence
+Prestwich
+Pracht
+Poutre
+Poucher
+Portera
+Polinsky
+Poage
+Platts
+Pineau
+Pinckard
+Pilson
+Pilling
+Pilkins
+Pili
+Pikes
+Pigram
+Pietila
+Pickron
+Pia
+Philippi
+Philhower
+Pflueger
+Pfalzgraf
+Pettibone
+Pett
+Petrosino
+Persing
+Perrino
+Perotti
+Periera
+Peri
+Peredo
+Peralto
+Pennywell
+Pennel
+Pen
+Pellegren
+Pella
+Pedroso
+Paulos
+Paulding
+Pates
+Pasek
+Paramo
+Paolino
+Panganiban
+Paneto
+Paluch
+Ozaki
+Ownbey
+Overfelt
+Outman
+Opper
+Onstad
+Oland
+Okuda
+Oertel
+Oelke
+Normandeau
+Nordby
+Nordahl
+Noecker
+Noblin
+No
+Niswonger
+Nishioka
+Nett
+Nephew
+Negley
+Needles
+Nedeau
+Natera
+Nachman
+Naas
+Musich
+Mungin
+Mourer
+Mounsey
+Mottola
+Mothershed
+Moskal
+Mosbey
+Morini
+Moreles
+Mood
+Montaluo
+Moneypenny
+Monda
+Moench
+Moates
+Moad
+Mixer
+Missildine
+Misiewicz
+Mirabella
+Minott
+Minnifield
+Mincks
+Milum
+Milani
+Mikelson
+Mestayer
+Mess
+Mertes
+Merrihew
+Merlos
+Meritt
+Melnyk
+Medlen
+Meder
+Mean
+Mcvea
+Mcquarrie
+Mcquain
+Mclucas
+Mclester
+Mckitrick
+Mckennon
+Mcinnes
+Mcgrory
+Mcgranahan
+Mcglamery
+Mcgivney
+Mcgilvray
+Mccuiston
+Mccuin
+Mccrystal
+Mccolley
+Mcclerkin
+Mcclenon
+Mccamey
+Mcaninch
+Mazariegos
+Maynez
+Mattioli
+Mastronardi
+Masone
+Marzett
+Marsland
+Mari
+Margulies
+Margolin
+Malatesta
+Malachi
+Mainer
+Maietta
+Magrath
+Maese
+Madkins
+Madeiros
+Madamba
+Mackson
+Mac
+Maben
+Lytch
+Lundgreen
+Lumb
+Lukach
+Luick
+Luetkemeyer
+Luechtefeld
+Ludy
+Ludden
+Luckow
+Lubinsky
+Lowes
+Lout
+Lorenson
+Loran
+Lopinto
+Looby
+Lones
+Livsey
+Liskey
+Lisby
+Lintner
+Lindow
+Lindblom
+Liming
+Liechty
+Leth
+Lesniewski
+Lenig
+Lemonds
+Leisy
+Lehrer
+Lehnen
+Lehmkuhl
+Leeth
+Leer
+Leeks
+Lechler
+Lebsock
+Lavere
+Lautenschlage
+Laughridge
+Lauderback
+Laudenslager
+Lassonde
+Laroque
+Laramee
+Laracuente
+Lapeyrouse
+Lampron
+Lamers
+Lamer
+Laino
+Lague
+Laguardia
+Lafromboise
+Lafata
+Lacount
+Lachowicz
+Kysar
+Kwiecien
+Kuffel
+Kueter
+Kronenberg
+Kristensen
+Kristek
+Krings
+Kriesel
+Krey
+Krebbs
+Kreamer
+Krabbe
+Kossman
+Kosakowski
+Kosak
+Kopacz
+Konkol
+Koepsell
+Koening
+Koen
+Knerr
+Knapik
+Kluttz
+Klocke
+Klenk
+Klemme
+Klapp
+Kitchell
+Kita
+Kissane
+Kirkbride
+Kirchhoff
+Kinter
+Kinsel
+Kingsland
+Kimmer
+Kimler
+Killoran
+Kieser
+Khalsa
+Khalaf
+Kettel
+Kerekes
+Keplin
+Kentner
+Kennebrew
+Kenison
+Kellough
+Kellman
+Keatts
+Keasey
+Kauppi
+Katon
+Kari
+Kanner
+Kampa
+Kall
+Kai
+Kaczorowski
+Kaczmarski
+Juarbe
+Jordison
+Jonathan
+Jobst
+Jezierski
+Jeanbart
+Jarquin
+Janey
+Jagodzinski
+Ishak
+Isett
+Isa
+Infantino
+Imburgia
+Illingworth
+Hysmith
+Hynson
+Hydrick
+Hurla
+Hunton
+Hunnell
+Humbertson
+Housand
+Hottle
+Hosch
+Hoos
+Honn
+Hohlt
+Hodel
+Hochmuth
+Hixenbaugh
+Hislop
+Hisaw
+Hintzen
+Hilgendorf
+Hilchey
+Higgens
+Hersman
+Herrara
+Hendrixson
+Hendriks
+Hemond
+Hemmingway
+Heminger
+Helgren
+Heisey
+Heilmann
+Hehn
+Hegna
+Heffern
+Hawrylak
+Haverty
+Hauger
+Haslem
+Harnett
+Harb
+Happ
+Hanzlik
+Hanway
+Hanby
+Hanan
+Hamric
+Hammaker
+Halas
+Hagenbuch
+Hacking
+Habeck
+Gwozdz
+Gutter
+Gunia
+Guise
+Guadarrama
+Grubaugh
+Grivas
+Griffieth
+Grieb
+Grewell
+Gregorich
+Grazier
+Graeber
+Graciano
+Gowens
+Goodpaster
+Gondek
+Gohr
+Goffney
+Godbee
+Gitlin
+Gisler
+Gin
+Gillyard
+Gillooly
+Gilchrest
+Gilbo
+Gierlach
+Giebler
+Giang
+Geske
+Gervasio
+Gertner
+Gehling
+Geeter
+Gaus
+Gattison
+Gatica
+Gathings
+Gath
+Gassner
+Gassert
+Garabedian
+Gamon
+Gameros
+Galban
+Gabourel
+Gaal
+Fuoco
+Fullenwider
+Fudala
+Friscia
+Franceschini
+Foronda
+Fontanilla
+Florey
+Florentino
+Flore
+Flegle
+Flecha
+Fisler
+Fischbach
+Fiorita
+Fines
+Figura
+Figgins
+Fichera
+Fester
+Ferra
+Fear
+Fawley
+Fawbush
+Fausett
+Farnes
+Farago
+Fairclough
+Fahie
+Fabiani
+Everest
+Evanson
+Eutsey
+Eshbaugh
+Esh
+Ertle
+Eppley
+Englehardt
+Engelhard
+Emswiler
+Elza
+Elling
+Elderkin
+Eland
+Efaw
+Edstrom
+Edmund
+Edgemon
+Ecton
+Echeverri
+Ebright
+Earheart
+Dynes
+Dygert
+Dyches
+Dulmage
+Duhn
+Duhamel
+Dues
+Dubrey
+Dubray
+Dubbs
+Drone
+Drey
+Drewery
+Dreier
+Dorval
+Dorough
+Dorais
+Donlin
+Donatelli
+Doke
+Dohm
+Doetsch
+Dobek
+Ditty
+Disbrow
+Ding
+Dinardi
+Dillahunty
+Dillahunt
+Diers
+Dier
+Diekmann
+Diangelo
+Deskin
+Deschaine
+Depaoli
+Denner
+Demyan
+Demont
+Demaray
+Delillo
+Deleeuw
+Deibel
+Decato
+Deblasio
+Debartolo
+Daubenspeck
+Darner
+Dardon
+Danziger
+Danials
+Damewood
+Dalpiaz
+Dallman
+Dallaire
+Cunniffe
+Cumpston
+Cumbo
+Cubero
+Cruzan
+Cronkhite
+Critelli
+Crimi
+Creegan
+Crean
+Craycraft
+Crater
+Cranfill
+Coyt
+Courchesne
+Coufal
+Corradino
+Corprew
+Colville
+Cocco
+Coby
+Clinch
+Clickner
+Clavette
+Claggett
+Cirigliano
+Ciesielski
+Christain
+Chesbro
+Chavera
+Chard
+Casteneda
+Castanedo
+Cast
+Casseus
+Casa
+Caruana
+Carnero
+Cappelli
+Capellan
+Canedy
+Cancro
+Camilleri
+Calero
+Cada
+Burghart
+Burbidge
+Bulfer
+Buis
+Budniewski
+Bucko
+Bruney
+Brugh
+Brossard
+Brodmerkel
+Brockmann
+Bring
+Brigmond
+Briere
+Bremmer
+Breck
+Breau
+Brautigam
+Brasch
+Brandenberger
+Bran
+Bragan
+Bozell
+Bowsher
+Bosh
+Borgia
+Borey
+Boomhower
+Bonneville
+Bonam
+Bolland
+Boise
+Boeve
+Boettger
+Boersma
+Boateng
+Bliven
+Blazier
+Blanca
+Blahnik
+Bjornstad
+Bitton
+Biss
+Birkett
+Billingsly
+Biagioni
+Bettle
+Bertucci
+Bertolino
+Bermea
+Bergner
+Berber
+Bensley
+Bendixen
+Beltrami
+Bellone
+Belland
+Bein
+Behringer
+Begum
+Beans
+Bayona
+Batiz
+Bassin
+Baskette
+Bartolomeo
+Bartolo
+Bartholow
+Barkan
+Barish
+Barett
+Bardo
+Bamburg
+Ballerini
+Balla
+Balis
+Bakley
+Bailon
+Bachicha
+Babiarz
+Ayars
+Axton
+Axel
+Awong
+Awe
+Awalt
+Auslander
+Ausherman
+Aumick
+Athens
+Atha
+Atchinson
+Aslett
+Askren
+Arrowsmith
+Arras
+Arnhold
+Armagost
+Arey
+Arcos
+Archibeque
+Antunes
+Antilla
+Ann
+Andras
+Amyx
+Amison
+Amero
+Alzate
+Alphonse
+Alper
+Aller
+Alioto
+Alexandria
+Aigner
+Agtarap
+Agbayani
+Adami
+Achorn
+Aceuedo
+Acedo
+Abundis
+Aber
+Abee
+Zuccaro
+Ziglar
+Zier
+Ziebell
+Zieba
+Zamzow
+Zahl
+Yurko
+Yurick
+Yonkers
+Yerian
+Yeaman
+Yarman
+Yann
+Yahn
+Yadon
+Yadao
+Woodbridge
+Wolske
+Wollenberg
+Wojtczak
+Wnuk
+Witherite
+Winther
+Winick
+Widell
+Wickens
+Whichard
+Wheelis
+Wesely
+Wentzell
+Wenthold
+Wemple
+Weisenburger
+Wehling
+Weger
+Weaks
+Water
+Wassink
+Warn
+Walquist
+Wadman
+Wacaster
+Waage
+Voliva
+Vlcek
+Villafana
+Vigliotti
+Viger
+Viernes
+Viands
+Vey
+Veselka
+Versteeg
+Vero
+Verhoeven
+Vendetti
+Velardo
+Vatter
+Vasconcellos
+Varn
+Vanwagner
+Vanvoorhis
+Vanhecke
+Vanduyn
+Vandervoort
+Vanderslice
+Valone
+Vallier
+Vails
+Uvalle
+Ursua
+Urenda
+Upright
+Uphoff
+Tustin
+Turton
+Turnbough
+Turck
+Tullio
+Tuch
+Truehart
+Tropea
+Troester
+Trippe
+Tricarico
+Trevarthen
+Trembly
+Trace
+Trabue
+Traber
+Toto
+Tosi
+Toal
+Tinley
+Tingler
+Timoteo
+Tiffin
+Tien
+Ticer
+Thurgood
+Thorman
+Therriault
+Theel
+Tessman
+Tekulve
+Tejera
+Tebbs
+Tavernia
+Tarpey
+Tallmadge
+Takemoto
+Szot
+Sylvest
+Swindoll
+Swearinger
+Swantek
+Swaner
+Swainston
+Susi
+Surrette
+Sur
+Supple
+Sullenger
+Sudderth
+Suddarth
+Suckow
+Strider
+Strege
+Stream
+Strassburg
+Stoval
+Stotz
+Stoneham
+Stilley
+Stille
+Stierwalt
+Stfleur
+Steuck
+Stermer
+Stclaire
+Stano
+Staker
+Stahler
+Stablein
+Srinivasan
+Squillace
+Sprvill
+Sproull
+Sprau
+Sporer
+Spore
+Spittler
+Speelman
+Sparr
+Sparkes
+Spang
+Spagnuolo
+Sosinski
+Sorto
+Sorkin
+Sondag
+Sollers
+Socia
+Snarr
+Smrekar
+Smolka
+Slyter
+Slovinsky
+Sliwa
+Slavik
+Slatter
+Skiver
+Skeem
+Skala
+Sitzes
+Sitsler
+Sitler
+Sinko
+Simser
+Siegler
+Sideris
+Shrewsberry
+Shoopman
+Shoaff
+Shira
+Shindler
+Shimmin
+Shill
+Shenkel
+Shemwell
+Shehorn
+Severa
+Sergio
+Semones
+Selsor
+Seller
+Sekulski
+Segui
+Sechrest
+Scot
+Schwer
+Schwebach
+Schur
+Schmiesing
+Schlick
+Schlender
+Schebler
+Schear
+Schapiro
+Sauro
+Saunder
+Sauage
+Satterly
+Saraiva
+Saracino
+Saperstein
+Sanmartin
+Sanluis
+Sandt
+Sandrock
+Sammet
+Sama
+Salk
+Sakata
+Saini
+Sackrider
+Rys
+Russum
+Russi
+Russaw
+Rozzell
+Roza
+Rowlette
+Rothberg
+Rossano
+Rosebrock
+Romanski
+Romanik
+Romani
+Roma
+Roiger
+Roig
+Roehr
+Rodenberger
+Rodela
+Rod
+Rochford
+Ristow
+Rispoli
+Ripper
+Rigo
+Riesgo
+Riebel
+Ribera
+Ribaudo
+Rhoda
+Reys
+Resendes
+Repine
+Reisdorf
+Reisch
+Rebman
+Rasmus
+Raske
+Ranum
+Rames
+Rambin
+Raman
+Rajewski
+Raffield
+Rady
+Radich
+Raatz
+Quinnie
+Pyper
+Puthoff
+Prow
+Proehl
+Pribyl
+Pretti
+Prete
+Presby
+Poyer
+Powelson
+Porteous
+Poquette
+Pooser
+Pollan
+Ploss
+Plewa
+Plants
+Placide
+Pion
+Pinnick
+Pinales
+Pin
+Pillot
+Pille
+Pilato
+Piggee
+Pietrowski
+Piermarini
+Pickford
+Piccard
+Phenix
+Pevey
+Petrowski
+Petrillose
+Pesek
+Perrotti
+Perfecto
+Peppler
+Peppard
+Penfold
+Pellitier
+Pelland
+Pehowic
+Pedretti
+Paules
+Passero
+Pasha
+Panza
+Pallante
+Palau
+Pakele
+Pacetti
+Paavola
+Overy
+Overson
+Outler
+Osegueda
+Ord
+Oplinger
+Oldenkamp
+Ok
+Ohern
+Oetting
+Odums
+Oba
+Nowlen
+Nowack
+Nordlund
+Noblett
+Nobbe
+Nierman
+Nichelson
+Niblock
+Newbrough
+Nest
+Nemetz
+Neeson
+Needleman
+Necessary
+Navin
+Nastasi
+Naslund
+Naramore
+Nakken
+Nakanishi
+Najarro
+Mushrush
+Muma
+Mulero
+Morganfield
+Moreman
+Morain
+Moquin
+Montrose
+Monterrosa
+Monsivais
+Monroig
+Monje
+Monfort
+Moises
+Moffa
+Moeckel
+Mobbs
+Mitch
+Misiak
+Mires
+Mirelez
+Mineo
+Mineau
+Milnes
+Mikeska
+Michelin
+Michalowski
+Meszaros
+Messineo
+Meshell
+Merten
+Meola
+Menton
+Mends
+Mende
+Memmott
+Melius
+Mehan
+Mcnickle
+Mcmorran
+Mclennon
+Mcleish
+Mclaine
+Mckendry
+Mckell
+Mckeighan
+Mcisaac
+Mcie
+Mcguinn
+Mcgillis
+Mcfatridge
+Mcfarling
+Mcelravy
+Mcdonalds
+Mcculla
+Mcconnaughy
+Mcconnaughey
+Mcchriston
+Mcbeath
+Mayr
+Matyas
+Matthiesen
+Matsuura
+Matinez
+Mathys
+Matarazzo
+Masker
+Masden
+Mascio
+Martis
+Marrinan
+Marinucci
+Margerum
+Marengo
+Manthe
+Mansker
+Manoogian
+Mankey
+Manigo
+Manier
+Mangini
+Mandelbaum
+Maltese
+Malsam
+Mallo
+Maliszewski
+Mainolfi
+Maharaj
+Maggart
+Magar
+Maffett
+Macmaster
+Macky
+Macdonnell
+Mable
+Lyvers
+Lyn
+Luzzi
+Lutman
+Luk
+Lover
+Lovan
+Lonzo
+Longest
+Longerbeam
+Lofthouse
+Loethen
+Lodi
+Llorens
+Lizardo
+Lizama
+Liz
+Litscher
+Lisowski
+Lipski
+Lipsett
+Lipkin
+Linzey
+Lineman
+Limerick
+Limb
+Limas
+Lige
+Lierman
+Liebold
+Liberti
+Leverton
+Levene
+Lesueur
+Lenser
+Lenker
+Lemme
+Legnon
+Lefrancois
+Ledwell
+Lavecchia
+Laurich
+Lauricella
+Latino
+Lannigan
+Landor
+Lamprecht
+Lamountain
+Lamore
+Lamonica
+Lammert
+Lamboy
+Lamarque
+Lamacchia
+Lalley
+Lagace
+Lacorte
+Lacomb
+Kyllonen
+Kyker
+Kye
+Kuschel
+Kupfer
+Kunde
+Kucinski
+Kubacki
+Kuan
+Kroenke
+Krech
+Koziel
+Kovacich
+Kothari
+Koth
+Kotek
+Kostelnik
+Kosloski
+Knoles
+Knabe
+Kmiecik
+Klingman
+Kliethermes
+Kleffman
+Klees
+Klaiber
+Kittell
+Kissling
+Kisinger
+Kintner
+Kinoshita
+Kiener
+Khouri
+Kerman
+Kelii
+Keirn
+Keezer
+Kaup
+Kathan
+Kaser
+Karlsen
+Kapur
+Kandoll
+Kammel
+Kahele
+Justesen
+Jue
+Jonason
+Johnsrud
+Joerling
+Jochim
+Jespersen
+Jeong
+Jenness
+Jedlicka
+Jakob
+Isaman
+Inghram
+Ingenito
+Imperial
+Iadarola
+Hynd
+Huxtable
+Huwe
+Huron
+Hurless
+Humpal
+Hughston
+Hughart
+Huggett
+Hugar
+Huether
+Howdyshell
+Houtchens
+Houseworth
+Hoskie
+Holshouser
+Holmen
+Holloran
+Hohler
+Hoefler
+Hodsdon
+Hochman
+Hjort
+Hippert
+Hippe
+Hinzman
+Hillock
+Hilden
+Hilde
+Heyn
+Heyden
+Heyd
+Hergert
+Henrikson
+Henningsen
+Hendel
+Helget
+Helf
+Helbing
+Heintzman
+Heggie
+Hege
+Hecox
+Heatherington
+Heare
+Haxton
+Haverstock
+Haverly
+Hatler
+Haselton
+Hase
+Hartzfeld
+Harten
+Harken
+Hargrow
+Haran
+Hanton
+Hammar
+Hamamoto
+Halper
+Halko
+Hackathorn
+Haberle
+Haake
+Gunnoe
+Gunkel
+Gulyas
+Guiney
+Guilbeau
+Guider
+Guerrant
+Gudgel
+Guarisco
+Grossen
+Grossberg
+Gropp
+Groome
+Grobe
+Gremminger
+Greenley
+Grauberger
+Grabenstein
+Gowers
+Gostomski
+Gosier
+Goodenow
+Gonzoles
+Goliday
+Goettle
+Goens
+Goates
+Glymph
+Glavin
+Glassco
+Gladys
+Gladfelter
+Glackin
+Githens
+Girgis
+Gimpel
+Gilbreth
+Gilbeau
+Giffen
+Giannotti
+Gholar
+Gervasi
+Gertsch
+Gernatt
+Gephardt
+Genco
+Gehr
+Geddis
+Gear
+Gase
+Garrott
+Garrette
+Gapinski
+Ganter
+Ganser
+Gangi
+Gangemi
+Gang
+Gallina
+Galdi
+Gailes
+Gaetano
+Gadomski
+Gaccione
+Fuschetto
+Furtick
+Furfaro
+Fullman
+Frutos
+Fruchter
+Frogge
+Freytag
+Freudenthal
+Fregoe
+Franzone
+Frankum
+Francia
+Franceschi
+Fraction
+Forys
+Forero
+Folkers
+Foil
+Flug
+Flitter
+Flemons
+Fitzer
+Firpo
+Finizio
+Filiault
+Figg
+Fiddler
+Fichtner
+Fetterolf
+Ferringer
+Feil
+Fayne
+Farro
+Faddis
+Ezzo
+Ezelle
+Eynon
+Evitt
+Eutsler
+Euell
+Escovedo
+Erne
+Eriksson
+Enriguez
+Empson
+Elkington
+Elk
+Eisenmenger
+Eidt
+Eichenberger
+Ehrmann
+Ediger
+Earlywine
+Eacret
+Duzan
+Dunnington
+Duffer
+Ducasse
+Dubiel
+Drovin
+Drager
+Drage
+Donham
+Donat
+Dona
+Dolinger
+Dokken
+Doepke
+Dodwell
+Docherty
+Distasio
+Disandro
+Diniz
+Digangi
+Didion
+Dezzutti
+Devora
+Detmer
+Deshon
+Derrigo
+Dentler
+Demoura
+Demeter
+Demeritt
+Demayo
+Demark
+Demario
+Delzell
+Delnero
+Delgrosso
+Dejarnett
+Debernardi
+Dearmas
+Dau
+Dashnaw
+Daris
+Danks
+Danker
+Dangler
+Daignault
+Dafoe
+Dace
+Curet
+Cumberledge
+Culkin
+Cuba
+Crowner
+Crocket
+Crawshaw
+Craun
+Cranshaw
+Cragle
+Courser
+Costella
+Cornforth
+Corkill
+Cordy
+Coopersmith
+Conzemius
+Connett
+Connely
+Condict
+Condello
+Concha
+Comley
+Colt
+Collen
+Cohoon
+Coday
+Clugston
+Clowney
+Clippard
+Clinkenbeard
+Clines
+Clelland
+Clause
+Clapham
+Clancey
+Clabough
+Cichy
+Cicalese
+Chuck
+Chua
+Chittick
+Chisom
+Chisley
+Chino
+Chinchilla
+Cheramie
+Cerritos
+Cercone
+Cena
+Cawood
+Cavness
+Catanzarite
+Casada
+Carvell
+Carp
+Carmicheal
+Carll
+Cardozo
+Caplin
+Candia
+Canby
+Cammon
+Callister
+Calligan
+Calkin
+Caillouet
+Buzzelli
+Bute
+Bustillo
+Bursey
+Burgeson
+Bupp
+Bulson
+Bulls
+Buist
+Buffey
+Buczkowski
+Buckbee
+Bucio
+Brueckner
+Broz
+Brookhart
+Brong
+Brockmeyer
+Broberg
+Brittenham
+Brisbois
+Bridgmon
+Bride
+Breyer
+Brede
+Breakfield
+Breakey
+Brauner
+Branigan
+Brandewie
+Branche
+Brager
+Brader
+Bovell
+Bouthot
+Bostock
+Bosma
+Boseman
+Boschee
+Borthwick
+Borneman
+Borer
+Borek
+Boomershine
+Boni
+Bommarito
+Bolman
+Boleware
+Boisse
+Boehlke
+Bodle
+Blash
+Blasco
+Blakesley
+Blacklock
+Blackley
+Bittick
+Birks
+Birdin
+Bircher
+Bilbao
+Bick
+Biby
+Bertoni
+Bertino
+Bertini
+Berson
+Bern
+Berkebile
+Bergstresser
+Benne
+Benevento
+Belzer
+Beltre
+Bellomo
+Bellerose
+Beilke
+Begeman
+Bebee
+Beazer
+Beaven
+Beamish
+Baymon
+Baston
+Bastidas
+Basom
+Basket
+Basey
+Bartles
+Baroni
+Barocio
+Barnet
+Barclift
+Banville
+Balthazor
+Balleza
+Balkcom
+Baires
+Bailiff
+Bailie
+Baik
+Baggott
+Bagen
+Bachner
+Babington
+Babel
+Asmar
+Askin
+Arvelo
+Artega
+Arrendondo
+Arreaga
+Arrambide
+Arquette
+Aronoff
+Arico
+Argentieri
+Arevalos
+Archbold
+Apuzzo
+Antczak
+Ankeny
+Angelle
+Angelini
+Anfinson
+Amer
+Amberg
+Amarillas
+Altier
+Altenburg
+Alspach
+Alosa
+Allsbrook
+Alexopoulos
+Aleem
+Aldred
+Albertsen
+Akerson
+Ainsley
+Agler
+Adley
+Addams
+Acoba
+Achille
+Abplanalp
+Abella
+Abare
+Zwolinski
+Zollicoffer
+Zola
+Zins
+Ziff
+Zenner
+Zender
+Zelnick
+Zelenka
+Zeches
+Zaucha
+Zauala
+Zappa
+Zangari
+Zagorski
+Youtsey
+Yorker
+Yell
+Yasso
+Yarde
+Yarbough
+Xiao
+Woolever
+Woodsmall
+Woodfolk
+Wonders
+Wobig
+Wixson
+Wittwer
+Wirtanen
+Winson
+Wingerd
+Wilkening
+Wilhelms
+Wierzbicki
+Wiechman
+Whites
+Weyrick
+Wessell
+Wenrick
+Wenning
+Weltz
+Weinrich
+Weiand
+Wehunt
+Wareing
+Walth
+Waibel
+Wahlquist
+Vona
+Voelkel
+Vitek
+Vinsant
+Vincente
+Vilar
+Viel
+Vicars
+Vermette
+Verma
+Vent
+Venner
+Veazie
+Vayda
+Vashaw
+Varon
+Vardeman
+Vandevelde
+Vanbrocklin
+Valery
+Val
+Vaccarezza
+Urquidez
+Urie
+Urbach
+Uram
+Ungaro
+Umali
+Ulsh
+Tutwiler
+Turnbaugh
+Tumminello
+Tuite
+Tueller
+Trulove
+Troha
+Trivino
+Trisdale
+Trippett
+Tribbett
+Treptow
+Tremain
+Travelstead
+Trautwein
+Trautmann
+Tram
+Traeger
+Tonelli
+Tomsic
+Tomich
+Tomasulo
+Tomasino
+Tole
+Todhunter
+Toborg
+Tischer
+Tirpak
+Tircuit
+Tinnon
+Tinnel
+Tines
+Tina
+Timbs
+Tilden
+Tiede
+Thumm
+Throne
+Throgmorton
+Thorndike
+Thornburgh
+Thoren
+Thomann
+Therrell
+Thau
+Thammavong
+Tetrick
+Tessitore
+Tesreau
+Teicher
+Teaford
+Tauscher
+Tauer
+Tanabe
+Talamo
+Takeuchi
+Taite
+Tadych
+Sweeton
+Swecker
+Swartzentrube
+Swarner
+Surrell
+Surbaugh
+Suppa
+Sunshine
+Sumbry
+Suchy
+Stuteville
+Studt
+Stromer
+Strome
+Streng
+Stonestreet
+Stockley
+Stmichel
+Sticker
+Stfort
+Sternisha
+Stensrud
+Steinhardt
+Steinback
+Steichen
+Stauble
+Stasiak
+Starzyk
+Stango
+Standerfer
+Stachowiak
+Springston
+Spratlin
+Spracklen
+Sponseller
+Spilker
+Spiegelman
+Spellacy
+Speiser
+Spaziani
+Spader
+Spackman
+Space
+Sorum
+Sopha
+Sollis
+Sollenberger
+Solivan
+Solheim
+Sokolsky
+Sogge
+Smyser
+Smitley
+Sloas
+Slinker
+Skora
+Skiff
+Skare
+Siverd
+Sivels
+Siska
+Siordia
+Simmering
+Simko
+Sime
+Silmon
+Silano
+Sieger
+Siebold
+Shukla
+Shreves
+Shoun
+Shortle
+Shonkwiler
+Shoals
+Shimmel
+Shiel
+Shieh
+Sherbondy
+Shenkman
+Shein
+Shearon
+Shean
+Shatz
+Shanholtz
+Shafran
+Shaff
+Shackett
+Sgroi
+Sewall
+Severy
+Sethi
+Sessa
+Sequra
+Sepulvado
+Seper
+Senteno
+Sendejo
+Semmens
+Seipp
+Segler
+Seegers
+Sedwick
+Sedore
+Sechler
+Sebastiano
+Scovel
+Scotton
+Scopel
+Schwend
+Schwarting
+Schutter
+Schrier
+Schons
+Scholtes
+Schnetzer
+Schnelle
+Schmutz
+Schlichter
+Schelling
+Schams
+Schamp
+Scarber
+Scallan
+Scalisi
+Scaffidi
+Saxby
+Sawrey
+Sauvageau
+Sauder
+Sarrett
+Sanzo
+Santizo
+Santella
+Santander
+Sandez
+Sandel
+Sammon
+Salsedo
+Salge
+Sailors
+Sagun
+Safi
+Sader
+Sacchetti
+Sablan
+Saber
+Saade
+Runnion
+Runkel
+Rung
+Rumbo
+Ruesch
+Ruegg
+Ruckle
+Ruchti
+Rubens
+Rubano
+Rozycki
+Roupe
+Roufs
+Rossel
+Rosmarin
+Rosero
+Rosenwald
+Roselle
+Ronca
+Romos
+Rolla
+Rohling
+Rohleder
+Roell
+Roehm
+Rochefort
+Roch
+Robotham
+Rivenburgh
+Riopel
+Riederer
+Ridlen
+Rias
+Rhudy
+Reynard
+Retter
+Respess
+Reppond
+Repko
+Rengifo
+Reinking
+Reichelt
+Reeh
+Redenius
+Rebolledo
+Raymundo
+Rauh
+Ratajczak
+Rapley
+Ranalli
+Ramie
+Raitt
+Radloff
+Radle
+Rabbitt
+Quay
+Quant
+Pusateri
+Puffinberger
+Puerta
+Provencio
+Proano
+Privitera
+Prenger
+Prellwitz
+Pousson
+Potier
+Poster
+Portz
+Portlock
+Porth
+Portela
+Portee
+Porchia
+Pollick
+Polinski
+Polfer
+Polanski
+Polachek
+Pluta
+Plourd
+Plauche
+Pitner
+Piontkowski
+Pileggi
+Pierotti
+Pico
+Piacente
+Phinisee
+Phaup
+Pfost
+Pettinger
+Pettet
+Petrich
+Peto
+Persley
+Persad
+Perlstein
+Perko
+Pere
+Penders
+Peifer
+Peco
+Pear
+Pay
+Pawley
+Pash
+Parrack
+Parady
+Papen
+Pangilinan
+Pandolfo
+Palone
+Palmertree
+Padin
+Ou
+Ottey
+Ottem
+Ostroski
+Ornstein
+Ormonde
+Onstott
+Oncale
+Oltremari
+Olcott
+Olan
+Oishi
+Oien
+Odonell
+Odonald
+Ode
+Obeso
+Obeirne
+Oatley
+Nusser
+Novo
+Novicki
+Noreen
+Nora
+Nitschke
+Nistler
+Nim
+Nikkel
+Niese
+Nierenberg
+Nield
+Niedzwiecki
+Niebla
+Niebel
+Nicklin
+Neyhart
+Newsum
+Nevares
+Nageotte
+Nagai
+Myung
+Mutz
+Murata
+Muralles
+Munnerlyn
+Mumpower
+Muegge
+Muckle
+Muchmore
+Moulthrop
+Motl
+Moskos
+Mortland
+Morring
+Mormile
+Morimoto
+Morikawa
+Morgon
+Mordecai
+Montour
+Mont
+Mongan
+Monell
+Miyasato
+Mish
+Minshew
+Mimbs
+Millin
+Milliard
+Mihm
+Middlemiss
+Miano
+Mew
+Mesick
+Merlan
+Mendonsa
+Mench
+Melonson
+Melling
+Mecca
+Meachem
+Mctighe
+Mcnelis
+Mcmurtrey
+Mcmurphy
+Mckesson
+Mckenrick
+Mckelvie
+Mcjunkins
+Mcgory
+Mcgirr
+Mcgeever
+Mcfield
+Mcelhinney
+Mccrossen
+Mccommon
+Mccannon
+Mazyck
+Mawyer
+Maull
+Matute
+Mathies
+Maschino
+Marzan
+Martinie
+Marrotte
+Marmion
+Markarian
+Marinacci
+Margolies
+Margeson
+Marcia
+Marcel
+Marak
+Maraia
+Maracle
+Manygoats
+Mano
+Manker
+Mank
+Mandich
+Manderson
+Maltz
+Malmquist
+Malacara
+Majette
+Mais
+Magnan
+Magliocca
+Madina
+Madara
+Macwilliams
+Macqueen
+Maccallum
+Lyde
+Lyday
+Lutrick
+Lurz
+Lurvey
+Lumbreras
+Luhrs
+Luhr
+Lue
+Lowrimore
+Lowndes
+Lowers
+Lourenco
+Lougee
+Lorona
+Longstreth
+Loht
+Lofquist
+Loewenstein
+Lobos
+Lizardi
+Liverpool
+Lionberger
+Limoli
+Liljenquist
+Liguori
+Liebl
+Liburd
+Leukhardt
+Letizia
+Lesinski
+Lepisto
+Lenzini
+Leisenring
+Leipold
+Leier
+Leggitt
+Legare
+Leaphart
+Lazor
+Lazaga
+Lavey
+Laue
+Laudermilk
+Lauck
+Lassalle
+Larsson
+Larison
+Lanzo
+Lantzy
+Lanners
+Langtry
+Landford
+Lancour
+Lamour
+Lambertson
+Lalone
+Lairson
+Lainhart
+Lagreca
+Lacina
+Labranche
+Labate
+Kurtenbach
+Kuipers
+Kuechle
+Kue
+Kubo
+Krinsky
+Krauser
+Kraeger
+Kracht
+Kozeliski
+Kozar
+Kowalik
+Kotler
+Kotecki
+Koslosky
+Kosel
+Koob
+Kolasinski
+Koizumi
+Kohlman
+Koffman
+Knutt
+Knore
+Knaff
+Kmiec
+Klamm
+Kittler
+Kitner
+Kirkeby
+Kiper
+Kindler
+Kilmartin
+Killings
+Killin
+Kilbride
+Kerchner
+Kendell
+Keddy
+Keaveney
+Kearsley
+Karras
+Karlsson
+Karalis
+Kappes
+Kapadia
+Kallman
+Kallio
+Kalil
+Kader
+Jurkiewicz
+Joya
+Johann
+Jitchaku
+Jillson
+Jex
+Jeune
+Jarratt
+Jarchow
+Janak
+Ivins
+Ivans
+Isenhart
+Inocencio
+Inoa
+Imhof
+Iacono
+Hynds
+Hutching
+Hutchin
+Hulsman
+Hulsizer
+Hueston
+Huddleson
+Hrbek
+Howry
+Housey
+Hounshell
+Hosick
+Hortman
+Horseman
+Horky
+Horine
+Hootman
+Honeywell
+Honeyestewa
+Holste
+Holien
+Holbrooks
+Hoffmeyer
+Hof
+Hoese
+Hoenig
+Hirschfeld
+Hildenbrand
+Higson
+Higney
+Hibert
+Hibbetts
+Hewlin
+Hesley
+Herrold
+Hermon
+Heritage
+Hepker
+Henwood
+Helbling
+Heinzman
+Heidtbrink
+Hedger
+Havey
+Hatheway
+Hartshorne
+Harpel
+Haning
+Handelman
+Hamalainen
+Hamad
+Halt
+Halasz
+Haigwood
+Haggans
+Hackshaw
+Guzzo
+Gunner
+Gundrum
+Guilbeault
+Gugliuzza
+Guglielmi
+Gue
+Guderian
+Gruwell
+Grunow
+Grundman
+Gruen
+Grotzke
+Grossnickle
+Groomes
+Grode
+Grochowski
+Grob
+Grein
+Greif
+Greenwall
+Greenup
+Grassl
+Grannis
+Grandfield
+Grames
+Grabski
+Grabe
+Gouldsberry
+Gotham
+Gosch
+Goody
+Goodling
+Goodermote
+Gonzale
+Golebiowski
+Goldson
+Godlove
+Glanville
+Gillin
+Gilkerson
+Giessler
+Giambalvo
+Giacomini
+Giacobbe
+Ghio
+Gergen
+Gentz
+Genrich
+Gelormino
+Gelber
+Geitner
+Geimer
+Gauthreaux
+Gaultney
+Garvie
+Gareau
+Garbo
+Garbacz
+Ganoe
+Gangwer
+Gandarilla
+Galyen
+Galt
+Galluzzo
+Gallon
+Galardo
+Gager
+Gaddie
+Gaber
+Gabehart
+Gaarder
+Fusilier
+Furnari
+Furbee
+Fugua
+Fruth
+Frohman
+Friske
+Frilot
+Fridman
+Frescas
+Freier
+Frayer
+Franzese
+Franklyn
+Frankenberry
+Frain
+Fosse
+Foresman
+Forbess
+Foot
+Florida
+Flook
+Fletes
+Fleer
+Fleek
+Fleegle
+Fishburne
+Fiscalini
+Finnigan
+Fini
+Filipiak
+Figueira
+Fiero
+Ficek
+Fiaschetti
+Ferren
+Ferrando
+Ferman
+Fergusson
+Fenech
+Feiner
+Feig
+Fees
+Faulds
+Fate
+Fariss
+Fantasia
+Falor
+Falke
+Ewings
+Eversley
+Everding
+Eunice
+Etling
+Essen
+Erskin
+Enstrom
+Enrico
+Engebretsen
+Ender
+Emma
+Eitel
+Eichberger
+Ehler
+Eekhoff
+Edrington
+Edmonston
+Edgmon
+Edes
+Eberlein
+Dwinell
+Dux
+Dupee
+Dunklee
+Dunk
+Dungey
+Dunagin
+Dumoulin
+Duggar
+Duenez
+Dudzic
+Dudenhoeffer
+Ducey
+Dub
+Drouillard
+Dreibelbis
+Dreger
+Dreesman
+Draughon
+Downen
+Double
+Dorminy
+Dominic
+Dombeck
+Dolman
+Doebler
+Dittberner
+Dishaw
+Disanti
+Dinicola
+Dinham
+Dimino
+Dilling
+Difrancesco
+Dicello
+Dibert
+Deshazer
+Deserio
+Descoteau
+Deruyter
+Dering
+Depinto
+Dente
+Demus
+Demattos
+Demarsico
+Delude
+Dekok
+Debrito
+Debois
+Deakin
+Dea
+Dayley
+Dawsey
+Dauria
+Datson
+Darty
+Darsow
+Darragh
+Darensbourg
+Dalleva
+Dalbec
+Dadd
+Cutcher
+Curb
+Cung
+Cuello
+Cuadros
+Crute
+Crutchley
+Crispino
+Crislip
+Crisco
+Crevier
+Creekmur
+Crance
+Cragg
+Crager
+Cozby
+Coyan
+Coxon
+Covalt
+Couillard
+Costley
+Costilow
+Cossairt
+Corvino
+Corigliano
+Cordaro
+Corbridge
+Corban
+Coor
+Cooler
+Conkel
+Cong
+Conary
+Coltrain
+Collopy
+Colgin
+Colen
+Colbath
+Coiro
+Coffie
+Cochrum
+Cobbett
+Clopper
+Cliburn
+Clendenon
+Clemon
+Clementi
+Clausi
+Cirino
+Cina
+Churn
+Churchman
+Chilcutt
+Cherney
+Cheetham
+Cheatom
+Chatelain
+Chandra
+Chalifour
+Cesa
+Cervenka
+Cerullo
+Cerreta
+Cerbone
+Cecchini
+Ceccarelli
+Cawthorn
+Cavalero
+Catalina
+Castner
+Castlen
+Castine
+Casimiro
+Casdorph
+Cartmill
+Cartmell
+Carro
+Carriger
+Carlee
+Carias
+Caravella
+Cappas
+Capen
+Cantey
+Canedo
+Camuso
+Camps
+Campanaro
+Camero
+Cambria
+Calzado
+Callejo
+Caligiuri
+Cafaro
+Cadotte
+Cacace
+Byrant
+Busbey
+Burtle
+Burres
+Burnworth
+Burggraf
+Burback
+Bunte
+Bunke
+Bulle
+Bugos
+Budlong
+Buckhalter
+Buccellato
+Brummet
+Bruff
+Brubeck
+Brouk
+Broten
+Brosky
+Broner
+Brittle
+Brislin
+Brimm
+Brillhart
+Bridgham
+Brideau
+Brennecke
+Brenna
+Breer
+Breeland
+Bredesen
+Branden
+Brackney
+Brackeen
+Boza
+Boyum
+Bowdry
+Bowdish
+Bouwens
+Bouvier
+Bougie
+Bouche
+Bottenfield
+Bostian
+Bossie
+Bosler
+Boschert
+Boroff
+Borello
+Boom
+Bonser
+Bonfield
+Bon
+Bole
+Boldue
+Bogacz
+Boemer
+Bluth
+Bloxom
+Blickenstaff
+Blessinger
+Bleazard
+Blatz
+Blanchet
+Blacksher
+Birchler
+Binning
+Binkowski
+Biltz
+Bilotta
+Bilagody
+Bigbee
+Bieri
+Biehle
+Bidlack
+Betker
+Bethers
+Bethell
+Bertha
+Bero
+Bernacchi
+Bermingham
+Berkshire
+Benvenuto
+Bensman
+Benoff
+Bencivenga
+Beman
+Bellow
+Bellany
+Belflower
+Belch
+Bekker
+Bejar
+Beisel
+Beichner
+Began
+Beedy
+Beas
+Beanblossom
+Bawek
+Baus
+Baugus
+Battie
+Battershell
+Bateson
+Basque
+Basford
+Bartone
+Barritt
+Barko
+Bann
+Bamford
+Baltrip
+Balon
+Balliew
+Ballam
+Baldus
+Ayling
+Avelino
+Ashwell
+Ashland
+Arseneau
+Arroyos
+Armendarez
+Arita
+Argust
+Archuletta
+Arcement
+Antonacci
+Anthis
+Antal
+Annan
+Andree
+Anderman
+Amster
+Amiri
+Amadon
+Alveraz
+Altomari
+Altmann
+Altenhofen
+Allers
+Allbee
+Allaway
+All
+Aleo
+Alcoser
+Alcorta
+Akhtar
+Ahuna
+Agramonte
+Agard
+Adkerson
+Achord
+Abt
+Abdi
+Abair
+Zurn
+Zoellner
+Zirk
+Zion
+Zee
+Zarro
+Zarco
+Zambo
+Zaiser
+Zaino
+Zachry
+Youd
+Yonan
+Yniguez
+Yepes
+Yeo
+Yellock
+Yellen
+Yeatts
+Yearling
+Yatsko
+Yannone
+Wyler
+Woodridge
+Wolfrom
+Wolaver
+Wolanin
+Wojnar
+Wojciak
+Wittmann
+Wittich
+Wiswell
+Wisser
+Wintersteen
+Wineland
+Willing
+Willford
+Wiginton
+Wigfield
+Wierman
+Wice
+Wiater
+Whitsel
+Whitbread
+Wheller
+Wettstein
+Werling
+Wente
+Wenig
+Wempe
+Welz
+Weinhold
+Weigelt
+Weichman
+Wedemeyer
+Weddel
+Ways
+Wayment
+Waycaster
+Wauneka
+Watzka
+Watton
+Warnell
+Warnecke
+Warmack
+Warder
+Wands
+Waldvogel
+Waldridge
+Wahs
+Wagganer
+Waddill
+Vyas
+Vought
+Votta
+Voiles
+Virga
+Viner
+Villella
+Villaverde
+Villaneda
+Viele
+Vickroy
+Vicencio
+Veve
+Vetere
+Vermilyea
+Verley
+Verburg
+Ventresca
+Veno
+Venard
+Venancio
+Velaquez
+Veenstra
+Vea
+Vasil
+Vanzee
+Vanwie
+Vantine
+Vant
+Vanschoyck
+Vannice
+Vankampen
+Vanicek
+Vandersloot
+Vanderpoel
+Vanderlinde
+Vallieres
+Uzzell
+Uzelac
+Uranga
+Uptain
+Updyke
+Uong
+Untiedt
+Umbrell
+Umbaugh
+Umbarger
+Ulysse
+Ullmann
+Ullah
+Tutko
+Turturro
+Turnmire
+Turnley
+Turcott
+Turbyfill
+Turano
+Tuminello
+Tumbleson
+Tsou
+Truscott
+Trulson
+Troutner
+Trone
+Troll
+Trinklein
+Tremmel
+Tredway
+Trease
+Traynham
+Traw
+Totty
+Torti
+Torregrossa
+Torok
+Tomkins
+Tomaino
+Tkach
+Tirey
+Tinsman
+Timpe
+Tiefenauer
+Tiedt
+Tidball
+Thwaites
+Thulin
+Throneburg
+Thorns
+Thorell
+Thorburn
+Thiemann
+Thieman
+Thesing
+Tham
+Terrien
+Terrance
+Telfair
+Taybron
+Tasson
+Tasso
+Tarro
+Tanenbaum
+Talent
+Tailor
+Taddeo
+Tada
+Taborn
+Tabios
+Szekely
+Szatkowski
+Sylve
+Swineford
+Swartzfager
+Swanton
+Swagerty
+Surrency
+Sunderlin
+Sumerlin
+Suero
+Suddith
+Sublette
+Stumpe
+Stueve
+Study
+Stuckert
+Strycker
+Struve
+Struss
+Strubbe
+Strough
+Strothmann
+Strahle
+Stoutner
+Stooksbury
+Stones
+Stonebarger
+Stokey
+Stoffer
+Stimmel
+Stief
+Stephans
+Stemper
+Steltenpohl
+Stellato
+Steinle
+Stegeman
+Steffler
+Steer
+Steege
+Steckman
+Stapel
+Stansbery
+Stanaland
+Stahley
+Stagnaro
+Stachowski
+Squibb
+Sprunger
+Sproule
+Sprehe
+Spreen
+Sprecher
+Sposato
+Spivery
+Souter
+Sopher
+Sommerfeldt
+Soffer
+Snowberger
+Snape
+Smylie
+Smyer
+Smack
+Slaydon
+Slatton
+Slaght
+Skovira
+Skeans
+Sjolund
+Sjodin
+Siragusa
+Singelton
+Sinatra
+Silis
+Siebenaler
+Shuffield
+Shobe
+Shiring
+Shimabukuro
+Shilts
+Sherley
+Sherbert
+Shelden
+Sheil
+Shedlock
+Shearn
+Shaub
+Sharbono
+Shapley
+Shands
+Shaheen
+Shaffner
+Servantez
+Sentz
+Seney
+Selin
+Seitzinger
+Seider
+Sehr
+Sego
+Segall
+Seeds
+Sebastien
+Scimeca
+Schwenck
+Schweiss
+Schwark
+Schwalbe
+Schucker
+Schronce
+Schrag
+Schouten
+Schoppe
+Schomaker
+Schnarr
+Schmied
+Schmader
+Schlicht
+Schlag
+Schield
+Schiano
+Scheve
+Scherbarth
+Schaumburg
+Schauman
+Scarpino
+Savinon
+Sassaman
+Sarah
+Saporito
+Sanville
+Santilli
+Santaana
+Sanda
+Salzmann
+Salman
+Saks
+Sagraves
+Safran
+Saccone
+Sa
+Rutty
+Russett
+Rupard
+Rump
+Rumbley
+Ruffins
+Ruacho
+Rozema
+Roxas
+Routson
+Rourk
+Rought
+Rotunda
+Rotermund
+Rosman
+Rosette
+Rork
+Rooke
+Rolin
+Rohm
+Rohlman
+Rohl
+Roeske
+Roecker
+Rober
+Robenson
+Riso
+Rinne
+Rima
+Riina
+Rigsbee
+Riggles
+Riester
+Rials
+Rhinehardt
+Reynaud
+Reyburn
+Rewis
+Revermann
+Reutzel
+Retz
+Rende
+Rendall
+Reistad
+Reinders
+Reichardt
+Rehrig
+Rehrer
+Recendez
+Reamy
+Raz
+Rauls
+Ratz
+Rattray
+Rasband
+Rapone
+Ragle
+Ragins
+Radican
+Raczka
+Rachels
+Raburn
+Rabren
+Raboin
+Ra
+Quesnell
+Quaintance
+Puccinelli
+Pruner
+Prouse
+Proud
+Prosise
+Proffer
+Prochazka
+Probasco
+Previte
+Prayer
+Pour
+Portell
+Porcher
+Popoca
+Poncho
+Pomroy
+Poma
+Polsky
+Polsgrove
+Polidore
+Podraza
+Plymale
+Plescia
+Pleau
+Platte
+Plato
+Pizzi
+Pinchon
+Picot
+Piccione
+Picazo
+Philibert
+Phebus
+Pfohl
+Petell
+Pesso
+Pesante
+Pervis
+Perrins
+Perley
+Perkey
+Pereida
+Penate
+Peloso
+Pellerito
+Peffley
+Peddicord
+Pecina
+Peale
+Peaks
+Payette
+Paxman
+Pawlikowski
+Pavy
+Pavlov
+Patry
+Patmon
+Patil
+Pater
+Patak
+Pasqua
+Pasche
+Partyka
+Parody
+Parmeter
+Pares
+Pardi
+Paonessa
+Pao
+Panozzo
+Panameno
+Paletta
+Pait
+Oyervides
+Ossman
+Oshima
+Ortlieb
+Orsak
+Orleans
+Onley
+On
+Oldroyd
+Okano
+Ohora
+Offley
+Oestreicher
+Odonovan
+Odham
+Odegard
+Obst
+Obriant
+Obrecht
+Nuccio
+Nowling
+Nowden
+Novelli
+Novell
+Nost
+Norstrom
+Norfolk
+Nordgren
+Nopper
+Noller
+Nisonger
+Niskanen
+Nienhuis
+Nienaber
+Neuwirth
+Neumeyer
+Neice
+Naugher
+Naiman
+Nagamine
+Mustin
+Murrietta
+Murdaugh
+Munar
+Mulberry
+Muhlbauer
+Mroczkowski
+Mowdy
+Mouw
+Mousel
+Mountcastle
+Moscowitz
+Mosco
+Morro
+Moresi
+Morago
+Moomaw
+Montroy
+Montpas
+Montieth
+Montanaro
+Mongelli
+Mon
+Mollison
+Mollette
+Moldovan
+Mohar
+Mizuno
+Mitchelle
+Mishra
+Misenheimer
+Minshall
+Minozzi
+Minniefield
+Minion
+Milhous
+Migliaccio
+Migdal
+Mickell
+Meyering
+Methot
+Mester
+Mesler
+Meriweather
+Mensing
+Mensah
+Menge
+Mendola
+Mendibles
+Meloche
+Melnik
+Mellas
+Meinert
+Mehrhoff
+Medas
+Meckler
+Mctague
+Mcspirit
+Mcshea
+Mcquown
+Mcquiller
+Mclarney
+Mckiney
+Mckearney
+Mcguyer
+Mcfarlan
+Mcfadyen
+Mcdanial
+Mcdanel
+Mccurtis
+Mccrohan
+Mccorry
+Mcclune
+Mccant
+Mccanna
+Mccandlish
+Mcaloon
+Mayall
+Maver
+Maune
+Matza
+Matty
+Matsuzaki
+Matott
+Mathey
+Mateos
+Masoner
+Masino
+Mas
+Marzullo
+Marz
+Maryland
+Marsolek
+Marquard
+Mario
+Marchetta
+Marberry
+Manzione
+Many
+Manthei
+Manka
+Mangram
+Mangle
+Mangel
+Mandato
+Mancillas
+Mammen
+Malina
+Maletta
+Malecki
+Majkut
+Mages
+Maestre
+Macphail
+Maco
+Macneill
+Macadam
+Lysiak
+Lyne
+Luxton
+Luptak
+Lundmark
+Luginbill
+Lovallo
+Louthan
+Lousteau
+Loupe
+Lotti
+Lopresto
+Lonsdale
+Longsworth
+Lohnes
+Loghry
+Logemann
+Lofaro
+Loeber
+Locastro
+Livings
+Litzinger
+Litts
+Liotta
+Lingard
+Lineback
+Lindy
+Lindhorst
+Lill
+Lide
+Lickliter
+Liberman
+Lewinski
+Levandowski
+Leimbach
+Leifer
+Leidholt
+Leiby
+Leibel
+Leibee
+Lehrke
+Lehnherr
+Lego
+Leese
+Leen
+Ledo
+Lech
+Leblond
+Leap
+Leahey
+Lazzari
+Lawrance
+Lawlis
+Lawhorne
+Lawes
+Lavigna
+Lavell
+Lauzier
+Lauter
+Laumann
+Latsha
+Latourette
+Latona
+Latney
+Laska
+Larner
+Larmore
+Larke
+Larence
+Lapier
+Lanzarin
+Lands
+Lammey
+Lamke
+Laminack
+Lamastus
+Lamaster
+Lacewell
+Labarr
+Laabs
+Kutch
+Kuper
+Kuna
+Kubis
+Krzemien
+Krupinski
+Krepps
+Kreeger
+Kraner
+Krammer
+Kountz
+Kothe
+Korpela
+Komara
+Kolenda
+Kolek
+Kohnen
+Koelzer
+Koelsch
+Kocurek
+Knoke
+Knauff
+Knaggs
+Knab
+Kluver
+Klose
+Klien
+Klahr
+Kitagawa
+Kissler
+Kirstein
+Kinnon
+Kinnebrew
+Kinnamon
+Kimmins
+Kilgour
+Kilcoyne
+Kiester
+Kiehm
+Kha
+Kesselring
+Kerestes
+Kenniston
+Kennamore
+Kenebrew
+Kelderman
+Keitel
+Kefauver
+Katzenberger
+Katt
+Kast
+Kassel
+Kasey
+Karol
+Kamara
+Kalmbach
+Kaizer
+Kaiwi
+Kainz
+Jurczyk
+Jumonville
+Juliar
+Jourdain
+Johndrow
+Johanning
+Johannesen
+Joffrion
+Jobes
+Jerde
+Jentzsch
+Jenkens
+Jendro
+Jellerson
+Jefferds
+Jaure
+Jaquish
+Janeway
+Jago
+Iwasaki
+Ishman
+Isaza
+Inmon
+Inlow
+Inclan
+Ildefonso
+Ike
+Iezzi
+Ianni
+Iacovetto
+Hyldahl
+Huxhold
+Huser
+Humpherys
+Humburg
+Hult
+Hullender
+Hulburt
+Huckabay
+Howeth
+Hovermale
+Hoven
+Houtman
+Hourigan
+Hosek
+Hopgood
+Homrich
+Holstine
+Holsclaw
+Hokama
+Hoffpauir
+Hoffner
+Hochstein
+Hochstatter
+Hochberg
+Hjelm
+Hiscox
+Hinsley
+Hinks
+Hineman
+Hineline
+Hinck
+Hilbun
+Hewins
+Herzing
+Hertzberg
+Hertenstein
+Herrea
+Herington
+Hercules
+Henrie
+Henman
+Hengst
+Hemmen
+Helmke
+Helgerson
+Heinsohn
+Heigl
+Hegstad
+Heggen
+Hegge
+Hefti
+Heathcock
+Haylett
+Haupert
+Haufler
+Hatala
+Haslip
+Hartless
+Hartje
+Hartis
+Harpold
+Harmsen
+Harbach
+Hanten
+Hanington
+Hammen
+Hameister
+Hallstrom
+Habersham
+Habegger
+Gussman
+Gundy
+Guitterez
+Guisinger
+Guilfoyle
+Groulx
+Grismer
+Griesbach
+Grawe
+Grall
+Graft
+Graben
+Goulden
+Gornick
+Gori
+Gookin
+Gonzalaz
+Gonyer
+Gonder
+Golphin
+Goller
+Goergen
+Glosson
+Glor
+Gladin
+Girdler
+Gillim
+Gillians
+Gillaspie
+Gilhooly
+Gildon
+Gignac
+Gibler
+Gibbins
+Giardino
+Giampietro
+Gettman
+Gerringer
+Gerrald
+Gerlich
+Georgiou
+Georgia
+Georgi
+Geiselman
+Gehman
+Gauze
+Gangl
+Gamage
+Gallian
+Gallen
+Gallatin
+Galen
+Galea
+Gainor
+Gahr
+Furbush
+Fulfer
+Fuhrmann
+Fritter
+Friis
+Friendly
+Friedly
+Freudenberger
+Frees
+Freemon
+Fratus
+Frans
+Foulke
+Fosler
+Forquer
+Fontan
+Folwell
+Folds
+Foeller
+Fodge
+Fobes
+Florek
+Fliss
+Flight
+Flesner
+Flegel
+Fitzloff
+Fiser
+First
+Firmin
+Firestine
+Finfrock
+Fineberg
+Figures
+Fiegel
+Fickling
+Fesperman
+Fernadez
+Felber
+Feimster
+Feazel
+Favre
+Faughn
+Fatula
+Fasone
+Farron
+Faron
+Farino
+Falvey
+Falkenberg
+Faley
+Faletti
+Faeth
+Fackrell
+Ezekiel
+Espe
+Eskola
+Escott
+Esaw
+Erps
+Erker
+Erath
+Enfield
+Emfinger
+Embury
+Embleton
+Emanuele
+Em
+Elvers
+Ellwanger
+Ellegood
+Einstein
+Eichinger
+Egge
+Egeland
+Edgett
+Echard
+Eblen
+Eastmond
+Duteau
+Durland
+Dure
+Dunlavy
+Dungee
+Dukette
+Dugay
+Duboise
+Dubey
+Dsouza
+Druck
+Dralle
+Doubek
+Dorta
+Dorch
+Dorce
+Dopson
+Dolney
+Dockter
+Distler
+Diss
+Dippel
+Diperna
+Dina
+Dichiara
+Dicerbo
+Dewindt
+Dewan
+Deveney
+Devargas
+Deutscher
+Deuel
+Detter
+Dess
+Derrington
+Deroberts
+Dern
+Deponte
+Denogean
+Denardi
+Denard
+Demary
+Demarcus
+Demarais
+Delucas
+Deloe
+Delmonico
+Delisi
+Delio
+Delduca
+Delaine
+Deihl
+Dehmer
+Deep
+Decoste
+Dechick
+Decatur
+Dec
+Debruce
+Debold
+Debell
+Deats
+Daunt
+Daquilante
+Dambrosi
+Damas
+Dalin
+Daisy
+Dahman
+Dahlem
+Daffin
+Dacquel
+Cutrell
+Cusano
+Curtner
+Currens
+Curnow
+Cuppett
+Cummiskey
+Cullers
+Culhane
+Crull
+Crossin
+Cropsey
+Cromie
+Crofford
+Criscuolo
+Crisafulli
+Crego
+Creeden
+Covello
+Covel
+Corse
+Correra
+Corners
+Cordner
+Cordier
+Coplen
+Copeman
+Contini
+Conteras
+Consalvo
+Conduff
+Condo
+Compher
+Comas
+Colliver
+Colan
+Cohill
+Cohenour
+Cogliano
+Codd
+Cockayne
+Clum
+Clowdus
+Clarida
+Clance
+Clairday
+Clagg
+Citron
+Citino
+Ciriello
+Cicciarelli
+Chrostowski
+Christley
+Christians
+Chrisco
+Chris
+Chrest
+Chisler
+Chieffo
+Cherne
+Cherico
+Cherian
+Cheirs
+Chauhan
+Charter
+Chamblin
+Cerra
+Cepero
+Cellini
+Celia
+Celeste
+Celedon
+Cejka
+Cavagnaro
+Cauffman
+Catanese
+Castrillo
+Castrellon
+Casserly
+Casino
+Caseres
+Carthen
+Carse
+Carragher
+Carpentieri
+Carmony
+Carmer
+Carlozzi
+Caradine
+Cappola
+Capece
+Capaldi
+Cantres
+Cantos
+Canevari
+Canete
+Calcaterra
+Cal
+Cadigan
+Cabbell
+Byrn
+Bykowski
+Butchko
+Busler
+Bushaw
+Buschmann
+Burow
+Buri
+Burgman
+Bunselmeyer
+Bunning
+Buhrman
+Budnick
+Buckson
+Buckhannon
+Brunjes
+Brummel
+Brumleve
+Bruckman
+Brouhard
+Brougham
+Brostrom
+Broerman
+Brocks
+Brison
+Brining
+Brindisi
+Brereton
+Breon
+Breitling
+Breedon
+Brasseaux
+Branaman
+Bramon
+Brackenridge
+Boyan
+Boxley
+Bouman
+Bouillion
+Botting
+Botti
+Bosshart
+Borup
+Borner
+Bordonaro
+Boot
+Bonsignore
+Bonsall
+Bolter
+Bojko
+Bohne
+Bohlmann
+Bogus
+Bogdon
+Boen
+Bodenschatz
+Bockoven
+Bobrow
+Blondin
+Blissett
+Bligen
+Blasini
+Blankenburg
+Bjorkman
+Bistline
+Bisset
+Birdow
+Biondolillo
+Bielski
+Biele
+Biddix
+Biddinger
+Bianchini
+Bevens
+Bevard
+Betancur
+Bernskoetter
+Bernet
+Bernardez
+Berliner
+Berland
+Berkheimer
+Berent
+Bensch
+Benesch
+Belleau
+Bedingfield
+Beckstrom
+Beckim
+Bechler
+Beachler
+Bazzell
+Basa
+Bartoszek
+Barsch
+Barrell
+Barnas
+Barnaba
+Barillas
+Barbier
+Baltodano
+Baltierra
+Balle
+Balint
+Baldi
+Balderson
+Balderama
+Baldauf
+Balcazar
+Balay
+Baiz
+Bairos
+Baba
+Azim
+Axe
+Aversa
+Avellaneda
+Ausburn
+Aurelio
+Auila
+Augusto
+Atwill
+Artiles
+Arterberry
+Aro
+Arnow
+Arnaud
+Arnall
+Armando
+Argyle
+Ares
+Arenz
+Arduini
+Archila
+Arakawa
+Appleman
+Aplin
+Antonini
+Anstey
+Anglen
+Andros
+Amweg
+Amstutz
+Amari
+Amadeo
+Aly
+Alteri
+Aloi
+Allebach
+Allah
+Aley
+Alamillo
+Airhart
+Ahrendt
+Africa
+Aegerter
+Adragna
+Admas
+Adderly
+Adderley
+Addair
+Abelar
+Abbamonte
+Abadi
+Zurek
+Zundel
+Zuidema
+Zuelke
+Zuck
+Zogg
+Zody
+Zets
+Zech
+Zecca
+Zavaleta
+Zarr
+Yousif
+Yoes
+Yoast
+Yeagley
+Yaney
+Yanda
+Yackel
+Wyles
+Wyke
+Woolman
+Woollard
+Woodis
+Woodin
+Wonderly
+Wombles
+Woloszyn
+Wollam
+Wnek
+Wms
+Wittie
+Withee
+Wissman
+Wisham
+Wintle
+Winthrop
+Winokur
+Winch
+Wilmarth
+Willhoite
+Wildner
+Wikel
+Wieser
+Wien
+Wicke
+Wiatrek
+Whitehall
+Whetstine
+Wheelus
+Weyrauch
+Weyers
+Westerling
+Wendelken
+Welner
+Welder
+Weinreb
+Weinheimer
+Weilbacher
+Weihe
+Weider
+Wecker
+Wead
+Watler
+Watkinson
+Wasmer
+Waskiewicz
+Wasik
+Warneke
+Wares
+Wangerin
+Wamble
+Walken
+Waker
+Wakeley
+Wahlgren
+Wahlberg
+Wagler
+Wachob
+Vorhies
+Vonseggern
+Vittitow
+Virgilio
+Vink
+Villarruel
+Villamil
+Villamar
+Villalovos
+Vidmar
+Victorero
+Vespa
+Vertrees
+Verissimo
+Veltman
+Vecchione
+Veals
+Varrone
+Varma
+Vanveen
+Vanterpool
+Vaneck
+Vandyck
+Vancise
+Vanausdal
+Vanalphen
+Valdiviezo
+Urton
+Urey
+Updegrove
+Unrue
+Ulbrich
+Tysinger
+Tyo
+Twiddy
+Tunson
+Trueheart
+Troyan
+Trier
+Traweek
+Trafford
+Tozzi
+Toulouse
+Touch
+Tosto
+Toste
+Torez
+Tooke
+Tonini
+Tonge
+Tomerlin
+Tolmie
+Tobe
+Tippen
+Tierno
+Tichy
+Thuss
+Threat
+Thran
+Thornbury
+Thone
+Theunissen
+Thelmon
+Theall
+Textor
+Teters
+Tesh
+Tennis
+Teng
+Tench
+Tekautz
+Tehrani
+Teat
+Teas
+Teare
+Te
+Tavenner
+Tartaglione
+Tanski
+Tanis
+Tanguma
+Tangeman
+Taney
+Tammen
+Tamburri
+Tamburello
+Talsma
+Tallie
+Takeda
+Taira
+Taheri
+Tademy
+Taddei
+Taaffe
+Szymczak
+Szczepaniak
+Szafranski
+Swygert
+Swem
+Swartzlander
+Sutley
+Supernaw
+Sundell
+Sullivant
+Suderman
+Sudbury
+Suares
+Stueber
+Stromme
+Striker
+Streeper
+Streck
+Strebe
+Stonehouse
+Stoia
+Stohr
+Stodghill
+Stirewalt
+Stick
+Sterry
+Stephanie
+Stenstrom
+Stene
+Steinbrecher
+Stear
+Stdenis
+Stanphill
+Staniszewski
+Stanard
+Stahlhut
+Stachowicz
+Srivastava
+Spong
+Spomer
+Spinosa
+Spindel
+Spera
+Spark
+Soward
+Sopp
+Sooter
+Sonnek
+Sonne
+Soland
+Sojourner
+Soeder
+Sobolewski
+Snellings
+Snare
+Smola
+Smetana
+Smeal
+Smarr
+Sloma
+Sligar
+Skenandore
+Skalsky
+Sitter
+Sissom
+Sirko
+Simkin
+Silverthorn
+Silman
+Sikkink
+Signorile
+Siddens
+Shumsky
+Shrider
+Shoulta
+Shonk
+Shomaker
+Shippey
+Shimada
+Shillingburg
+Shifflet
+Shiels
+Shepheard
+Sheerin
+Shedden
+Sheckles
+Sharrieff
+Sharpley
+Shappell
+Shaneyfelt
+Shampine
+Shaefer
+Shaddock
+Shadd
+Sforza
+Severtson
+Setzler
+Sepich
+Senne
+Senatore
+Sementilli
+Selway
+Selover
+Sellick
+Seigworth
+Sefton
+Seegars
+Sebourn
+Seaquist
+Sealock
+Seabreeze
+Scriver
+Scinto
+Schumer
+Schulke
+Schryver
+Schriner
+Schramek
+Schoon
+Schoolfield
+Schonberger
+Schnieder
+Schnider
+Schlitz
+Schlather
+Schirtzinger
+Scherman
+Schenker
+Scheiner
+Scheible
+Schaus
+Schakel
+Schaad
+Saxe
+Savely
+Savary
+Sardinas
+Santarelli
+Sanschagrin
+Sans
+Sanpedro
+Sanjose
+Sandra
+Sandine
+Sandigo
+Sandgren
+Sanderford
+Sandahl
+Salzwedel
+Salzar
+Salvino
+Salvatierra
+Salminen
+Salierno
+Salberg
+Sahagun
+Saelee
+Sabel
+Rynearson
+Ryker
+Rupprecht
+Runquist
+Rumrill
+Ruhnke
+Rovira
+Rottenberg
+Rosoff
+Rosete
+Rosebrough
+Roppolo
+Roope
+Romas
+Roley
+Rohrback
+Rohlfs
+Rogriguez
+Roel
+Rodriguiz
+Rodewald
+Roback
+Rizor
+Ritt
+Rippee
+Riolo
+Rinkenberger
+Riggsby
+Rigel
+Rieman
+Riedesel
+Rideau
+Ricke
+Rhinebolt
+Rheault
+Revak
+Relford
+Reinsmith
+Reichmann
+Rei
+Regula
+Redlinger
+Redhead
+Rayno
+Raycroft
+Rave
+Raus
+Raupp
+Rathmann
+Rastorfer
+Rasey
+Raponi
+Rantz
+Ranno
+Ranes
+Randal
+Ramp
+Ramnauth
+Rahal
+Raddatz
+Quattrocchi
+Quang
+Purchase
+Pullis
+Pulanco
+Pryde
+Prohaska
+Primiano
+Prez
+Prevatt
+Prechtl
+Pottle
+Potenza
+Portes
+Porowski
+Poppleton
+Pontillo
+Pong
+Polka
+Politz
+Politi
+Poggi
+Plonka
+Plaskett
+Placzek
+Pizzuti
+Pizzaro
+Pisciotta
+Pippens
+Pinkins
+Pinilla
+Pini
+Pingitore
+Piercey
+Pickup
+Piccola
+Piccioni
+Picciano
+Phy
+Philps
+Philp
+Philo
+Philmon
+Philbin
+Pflieger
+Pezzullo
+Petruso
+Petrea
+Petitti
+Peth
+Peshlakai
+Peschel
+Persico
+Persichetti
+Persechino
+Perris
+Perlow
+Perico
+Pergola
+Penniston
+Pembroke
+Pellman
+Pekarek
+Peirson
+Pearcey
+Pealer
+Pavlicek
+Passino
+Pasquarello
+Pasion
+Parzych
+Parziale
+Parga
+Papalia
+Papadakis
+Paino
+Pacini
+Oyen
+Ownes
+Owczarzak
+Outley
+Ouelette
+Ottosen
+Otting
+Ostwinkle
+Osment
+Oshita
+Osario
+Orlow
+Oriordan
+Orefice
+Orantes
+Oran
+Orahood
+Opel
+Olpin
+Oliveria
+Okon
+Okerlund
+Okazaki
+Ohta
+Offerman
+Nyce
+Nutall
+Northey
+Norcia
+Noor
+Noh
+Niehoff
+Niederhauser
+Nickolson
+Nguy
+Neylon
+Newstrom
+Nevill
+Netz
+Nesselrodt
+Nemes
+Neally
+Nauyen
+Nascimento
+Nardella
+Nanni
+Myren
+Murchinson
+Munter
+Munster
+Mundschenk
+Mujalli
+Muckleroy
+Mu
+Moussa
+Mouret
+Moulds
+Mottram
+Motte
+Mosey
+Morre
+Montreuil
+Monton
+Montellano
+Monninger
+Monhollen
+Mongeon
+Monestime
+Monegro
+Mondesir
+Monceaux
+Mola
+Moga
+Moening
+Moccia
+Misko
+Miske
+Mishaw
+Minturn
+Mingione
+Minerva
+Milstein
+Milos
+Milla
+Milks
+Milhouse
+Michl
+Micheletti
+Michals
+Mesia
+Merson
+Meras
+Menifee
+Meluso
+Mella
+Melick
+Mehlman
+Meffert
+Medoza
+Mecum
+Meaker
+Meahl
+Mczeal
+Mcwatters
+Mcomber
+Mcmonigle
+Mckiddy
+Mcgranor
+Mcgeary
+Mcgaw
+Mcenery
+Mcelderry
+Mcduffey
+Mccuistion
+Mccrudden
+Mccrossin
+Mccosh
+Mccolgan
+Mcclish
+Mcclenahan
+Mcclam
+Mccartt
+Mccarrell
+Mcbane
+Mc
+Maybury
+Mayben
+Maw
+Maulden
+Mauceri
+Matko
+Mathie
+Matheis
+Mathai
+Masucci
+Massiah
+Martorano
+Martnez
+Martindelcamp
+Marschke
+Marovich
+Markiewicz
+Marinaccio
+Marhefka
+Marcrum
+Manton
+Mantel
+Mannarino
+Manlove
+Mangham
+Manasco
+Malpica
+Mallernee
+Malinsky
+Malhotra
+Maish
+Maisel
+Mainville
+Maharrey
+Magid
+Maertz
+Mada
+Maclaughlin
+Macina
+Macdermott
+Macallister
+Macadangdang
+Maack
+Lynk
+Lydic
+Luyando
+Lutke
+Lupinacci
+Lunz
+Lundsten
+Lull
+Lujano
+Luhn
+Luecke
+Luebbe
+Ludolph
+Luckman
+Lucker
+Luckenbill
+Luckenbach
+Lucido
+Lowney
+Lowitz
+Lovaglio
+Louro
+Louk
+Loudy
+Louderback
+Lorick
+Lorenzini
+Lorensen
+Lorenc
+Lomuscio
+Loguidice
+Lockner
+Lockart
+Lochridge
+Litaker
+Lisowe
+Liptrap
+Linnane
+Linhares
+Lindfors
+Lindenmuth
+Lincourt
+Lina
+Like
+Liew
+Lies
+Liebowitz
+Levengood
+Leskovec
+Lesch
+Leoni
+Lennard
+Legner
+Leaser
+Leas
+Lean
+Leadingham
+Lazarski
+Layland
+Laurito
+Laulu
+Laughner
+Laughman
+Laughery
+Laube
+Latiolais
+Lasserre
+Lasser
+Lars
+Larrow
+Larrea
+Lapsley
+Lantrip
+Lanthier
+Langwell
+Langelier
+Landaker
+Lampi
+Lamond
+Lamblin
+Lambie
+Lakins
+Laipple
+Lagrimas
+Lafrancois
+Laffitte
+Laday
+Lacko
+Lacava
+Labor
+Labianca
+Kutsch
+Kuske
+Kunert
+Kubly
+Kuamoo
+Krummel
+Krise
+Krenek
+Kreiser
+Krausz
+Kraska
+Krakowski
+Kradel
+Kozik
+Koza
+Kotowski
+Koslow
+Korber
+Kojima
+Kochel
+Knabjian
+Klunder
+Klugh
+Klinkhammer
+Kliewer
+Klever
+Kleber
+Klages
+Klaas
+Kizziar
+Kitchel
+Kishimoto
+Kirschenman
+Kirschenbaum
+Kinnick
+Kinn
+Kinkle
+Kiner
+Kindla
+Kindall
+Kincaide
+Kilson
+Killins
+Kill
+Kightlinger
+Kienzle
+Kiah
+Khim
+Ketcherside
+Kerl
+Kelsoe
+Kelker
+Keizer
+Keir
+Keepers
+Kawano
+Kawa
+Kaveney
+Kath
+Kasparek
+Kaplowitz
+Kantrowitz
+Kant
+Kanoff
+Kano
+Kann
+Kamalii
+Kalt
+Kaleta
+Kalbach
+Kalauli
+Kalata
+Kalas
+Kaigler
+Kachel
+Juran
+Jubb
+Jonker
+Jonke
+Jolivette
+Joles
+Joas
+Jividen
+Jewel
+Jeffus
+Jeanty
+Jarvi
+Jardon
+Janvier
+Janosko
+Janoski
+Janiszewski
+Janish
+Janek
+Iwanski
+Iuliano
+Isabella
+Irle
+Ingmire
+Imber
+Ijames
+Iiams
+Ihrig
+Ichikawa
+Hynum
+Hutzel
+Hutts
+Huskin
+Husak
+Hurndon
+Huntsinger
+Humm
+Hulette
+Huitron
+Huguenin
+Hugg
+Hugee
+Huelskamp
+Huch
+Howen
+Hovanec
+Hoston
+Hostettler
+Horsfall
+Horodyski
+Holzhauer
+Hollimon
+Hollender
+Hogarth
+Hoffelmeyer
+Histand
+Hissem
+Hisel
+Hirayama
+Hinegardner
+Hinde
+Hinchcliffe
+Hiltbrand
+Hilsinger
+Hillstrom
+Hiley
+Hickenbottom
+Hickam
+Hibley
+Heying
+Hewson
+Hetland
+Hersch
+Herlong
+Herda
+Henzel
+Henshall
+Hendler
+Hence
+Helson
+Helfen
+Heinbach
+Heikkila
+Heggs
+Hefferon
+Hebard
+Heathcote
+Hearl
+Heaberlin
+Hauth
+Hauschild
+Haughney
+Hauch
+Hattori
+Haste
+Hasley
+Hartpence
+Harroun
+Harrier
+Harelson
+Hardgrove
+Hardel
+Hansbrough
+Handsome
+Handshoe
+Handly
+Haluska
+Hally
+Halling
+Halfhill
+Halferty
+Hakanson
+Haist
+Hairgrove
+Hahner
+Hagg
+Hafele
+Haaland
+Guttierez
+Gutknecht
+Gunnarson
+Gunlock
+Gummersheimer
+Gullatte
+Guity
+Guilmette
+Guhl
+Guenette
+Guardino
+Groshong
+Grober
+Gripp
+Grillot
+Grilli
+Greulich
+Gretzinger
+Greenwaldt
+Graven
+Grassman
+Granberg
+Graeser
+Graeff
+Graef
+Grabow
+Grabau
+Gotchy
+Goswick
+Gosa
+Gordineer
+Gorczyca
+Goodchild
+Golz
+Gollihue
+Goldwire
+Goldbach
+Goffredo
+Glassburn
+Glaeser
+Gillilan
+Gigante
+Giere
+Gieger
+Gidcumb
+Giarrusso
+Giannelli
+Gettle
+Gesualdi
+Geschke
+Gerwig
+Gervase
+Geoffrion
+Gentilcore
+Genther
+Gemes
+Gemberling
+Gelles
+Geitz
+Geeslin
+Gedney
+Gebauer
+Gaye
+Gawron
+Gavia
+Gautney
+Gaustad
+Gasmen
+Gargus
+Ganske
+Ganger
+Galvis
+Gallinger
+Gallichio
+Galletta
+Gaede
+Gadlin
+Gaby
+Gabrielsen
+Gaboriault
+Furlan
+Furgerson
+Fujioka
+Fugett
+Fuehrer
+Frisco
+Frint
+Frigon
+Frevert
+Frautschi
+Fraker
+Fradette
+Foulkes
+Forslund
+Forni
+Foo
+Fontenette
+Fones
+Folz
+Folmer
+Follman
+Folkman
+Flourney
+Flickner
+Flemmings
+Fleischacker
+Flander
+Flament
+Fithian
+Fister
+Fiorello
+Fiorelli
+Fioravanti
+Fieck
+Ficke
+Fiallos
+Fiacco
+Feuer
+Ferrington
+Fernholz
+Feria
+Fergurson
+Feick
+Febles
+Favila
+Faulkingham
+Fath
+Farnam
+Falter
+Fakhouri
+Fairhurst
+Failing
+Fahs
+Eva
+Estrello
+Essick
+Espree
+Esmond
+Eskelson
+Escue
+Escatel
+Erebia
+Epperley
+Epler
+Enyart
+Engelbert
+Enderson
+Emmitt
+Emch
+Elisondo
+Eli
+Elford
+El
+Ekman
+Eick
+Eichmann
+Ehrich
+Ehlen
+Edwardson
+Edley
+Edghill
+Edel
+Eastes
+Easterbrooks
+Eagleson
+Eagen
+Eade
+Dyle
+Dutkiewicz
+Dunnagan
+Duncil
+Duling
+Drumgoole
+Droney
+Dreyfus
+Dragan
+Dowty
+Doscher
+Dornan
+Doremus
+Doogan
+Donaho
+Donahey
+Dombkowski
+Dolton
+Dolen
+Dobratz
+Diveley
+Dittemore
+Ditsch
+Disque
+Dishmon
+Disch
+Dirickson
+Dippolito
+Dimuccio
+Dilger
+Diefenderfer
+Dicola
+Diblasio
+Dibello
+Devan
+Dettmer
+Deschner
+Desbiens
+Derusha
+Denkins
+Demonbreun
+Demchak
+Delucchi
+Delprete
+Deloy
+Deliz
+Deline
+Delap
+Deiter
+Deignan
+Degiacomo
+Degaetano
+Defusco
+Dede
+Deboard
+Debiase
+Deaville
+Deadwyler
+Davanzo
+Daughton
+Darter
+Darrin
+Danser
+Dandrade
+Dando
+Dampeer
+Dalziel
+Dalen
+Dain
+Dai
+Dague
+Czekanski
+Cutwright
+Cutliff
+Curle
+Cuozzo
+Cunnington
+Cunning
+Cunnigham
+Cumings
+Crowston
+Croak
+Crittle
+Crispell
+Crisostomo
+Crear
+Creach
+Craigue
+Crabbs
+Cozzi
+Cozza
+Coxe
+Cowsert
+Coviello
+Couse
+Coull
+Cottier
+Costagliola
+Corra
+Corpening
+Cormany
+Corless
+Corkern
+Conteh
+Conquest
+Conkey
+Cones
+Conditt
+Conaty
+Colomb
+Collura
+Colledge
+Colins
+Colgate
+Coleson
+Colemon
+Coins
+Coffland
+Coccia
+Coast
+Clougherty
+Clewell
+Cleckley
+Cleaveland
+Clarno
+Clamp
+Civils
+Cillo
+Cifelli
+Ciesluk
+Chum
+Chui
+Christison
+Christiana
+Chowning
+Chouteau
+Choung
+Childres
+Cherrington
+Chenette
+Cheeves
+Cheairs
+Chaddock
+Cernoch
+Cerino
+Cazier
+Cathy
+Castel
+Casselberry
+Caserta
+Carvey
+Carton
+Cart
+Carry
+Carris
+Carrie
+Carmant
+Cariello
+Cardarelli
+Caras
+Caracciolo
+Capitano
+Cantoni
+Cantave
+Cancio
+Campillo
+Cam
+Callens
+Caldero
+Calamia
+Cahee
+Cahan
+Cahalan
+Cabanilla
+Cabal
+Bywater
+Bynes
+Byassee
+Butkus
+Busker
+Bushby
+Busack
+Burtis
+Burrola
+Buroker
+Burnias
+Burn
+Burlock
+Burham
+Burak
+Bulla
+Buffin
+Buffa
+Buening
+Budney
+Buchannan
+Buchalter
+Bua
+Brule
+Brugler
+Broxson
+Broun
+Brosh
+Brissey
+Brisby
+Brinlee
+Brinkmeyer
+Brimley
+Brickell
+Breth
+Breger
+Brees
+Brank
+Braker
+Bozak
+Bowlds
+Bowersock
+Bousman
+Boushie
+Botz
+Bordwell
+Bonkowski
+Bonine
+Bonifay
+Bonesteel
+Boldin
+Bohringer
+Bohlander
+Boecker
+Bocook
+Bocock
+Boblett
+Bobbett
+Boas
+Boarman
+Bleser
+Blazejewski
+Blaustein
+Blausey
+Blancarte
+Blaize
+Blackson
+Blacketer
+Blackard
+Bisch
+Birchett
+Billa
+Bilder
+Bierner
+Bienvenu
+Bielinski
+Bialas
+Biagini
+Beynon
+Beyl
+Bettini
+Bethany
+Betcher
+Bessent
+Beshara
+Besch
+Bernd
+Bergemann
+Bergeaux
+Berdan
+Bens
+Benedicto
+Bendall
+Beltron
+Beltram
+Bellville
+Beisch
+Behney
+Beemer
+Beechler
+Beckum
+Becks
+Batzer
+Batte
+Bastida
+Bassette
+Basley
+Base
+Bartosh
+Bartolone
+Barraclough
+Barnick
+Barket
+Barkdoll
+Baringer
+Barges
+Barella
+Barbian
+Barbati
+Bannan
+Banderas
+Balles
+Baldo
+Balasubramani
+Bala
+Baig
+Bahn
+Bachmeier
+Babyak
+Baas
+Baars
+Ayuso
+Axt
+Avinger
+Avella
+Ausbrooks
+Aull
+Augello
+Atkeson
+Atkerson
+Atherley
+Athan
+Assad
+Asebedo
+Arrison
+Armon
+Armfield
+Armbrust
+Arlington
+Arkin
+Archambeau
+Antonellis
+Angotti
+Andy
+Amorose
+Amini
+Amborn
+Amano
+Aluarez
+Alma
+Allgaier
+Allegood
+Ales
+Alen
+Aldama
+Albertine
+Aki
+Aird
+Ahsing
+Ahmann
+Aguado
+Agostino
+Agostinelli
+Agnes
+Adwell
+Adsit
+Adelstein
+Ade
+Actis
+Acierno
+Achee
+Abbs
+Abbitt
+Zwagerman
+Zuercher
+Zinno
+Zettler
+Zeff
+Zavalza
+Zaugg
+Zarzycki
+Zappulla
+Zanotti
+Zachman
+Zacher
+Yundt
+Yslas
+Younes
+Yontz
+Yglesias
+Yeske
+Yellow
+Yeargin
+Yauger
+Yamane
+Xang
+Wylam
+Wrobleski
+Wratchford
+Worker
+Woodlee
+Wolsey
+Wolfinbarger
+Wohlenhaus
+Wittler
+Wittenmyer
+Witkop
+Wishman
+Wintz
+Winkelmann
+Windus
+Winborn
+Wims
+Wiltrout
+Wilshire
+Willmott
+Williston
+Wilemon
+Wilbourne
+Wiedyk
+Widmann
+Wickland
+Wickes
+Wichert
+Whitsell
+Whisenand
+Whidby
+Wetz
+Westmeyer
+Wertheim
+Wernert
+Werle
+Werkheiser
+Weng
+Weldin
+Weissenborn
+Weingard
+Weinfeld
+Weihl
+Weightman
+Weichel
+Wehrheim
+Wegrzyn
+Wegmann
+Wearing
+Waszak
+Wankum
+Wangler
+Walthour
+Waltermire
+Walstad
+Waldren
+Walbert
+Walawender
+Wahlund
+Wahlert
+Wahlers
+Wach
+Vuncannon
+Vroom
+Vredenburgh
+Vonk
+Vollmar
+Voisinet
+Vlahos
+Viscardi
+Vires
+Vipperman
+Violante
+Vidro
+Vessey
+Vesper
+Veron
+Vergari
+Verbeck
+Venturino
+Velastegui
+Vegter
+Varas
+Vanwey
+Vanvranken
+Vanvalkenbur
+Vanorsdale
+Vanoli
+Vanochten
+Vanier
+Vanevery
+Vane
+Vanduser
+Vandersteen
+Vandell
+Vandall
+Vallot
+Vallon
+Vallez
+Vallely
+Vadenais
+Uthe
+Usery
+Unga
+Ultsch
+Ullom
+Tyminski
+Twogood
+Tursi
+Turay
+Tungate
+Truxillo
+Trulock
+Trovato
+Troise
+Tripi
+Trinks
+Trimboli
+Trickel
+Trezise
+Trefry
+Treen
+Trebilcock
+Travieso
+Trachtenberg
+Touhey
+Tougas
+Tortorella
+Tormey
+Torelli
+Torborg
+Toran
+Tomek
+Tomassi
+Tollerson
+Tolden
+Toda
+Tobon
+Tjelmeland
+Titmus
+Tilbury
+Tietje
+Thurner
+Thum
+Thrope
+Thornbrough
+Thibaudeau
+Thackeray
+Tesoro
+Territo
+Ternes
+Teich
+Tecson
+Teater
+Teagarden
+Tatsch
+Tarallo
+Tapanes
+Tanberg
+Tamm
+Sylvis
+Swenor
+Swedlund
+Swagger
+Sutfin
+Sura
+Sundt
+Sundin
+Summerson
+Sumatzkuku
+Sultemeier
+Sulivan
+Suggitt
+Suermann
+Sturkie
+Sturgess
+Stumph
+Stuemke
+Struckhoff
+Strose
+Stroder
+Stride
+Stricklen
+Strick
+Streib
+Strei
+Strawther
+Stratis
+Strahm
+Stortz
+Storrer
+Storino
+Stohler
+Stohl
+Stockel
+Stinnette
+Stile
+Stieber
+Stensland
+Steffenhagen
+Stefanowicz
+Steever
+Steagall
+Statum
+Stapley
+Stanish
+Standiford
+Standen
+Stamos
+Stahlecker
+Stadtler
+Spratley
+Spraker
+Sposito
+Spickard
+Spehar
+Spees
+Spearing
+Spangle
+Spallone
+Sox
+Soulard
+Sorel
+Sora
+Sopko
+Sood
+Sonnen
+Som
+Solly
+Solesbee
+Soldano
+Sobey
+Sobczyk
+Snedegar
+Sneddon
+Smolinski
+Smolik
+Slota
+Sloman
+Sleigh
+Slavick
+Skorupski
+Skolnik
+Skirvin
+Skeels
+Skains
+Skahan
+Skaar
+Siwiec
+Siverly
+Siver
+Sivak
+Sirk
+Sinton
+Sinor
+Sincell
+Silberstein
+Sieminski
+Sidelinger
+Shurman
+Shunnarah
+Shirer
+Shidler
+Sherlin
+Shepperson
+Shemanski
+Sharum
+Shartrand
+Shapard
+Shanafelt
+Shamp
+Shader
+Shackelton
+Seyer
+Seroka
+Sernas
+Seright
+Serano
+Sengupta
+Semper
+Selinger
+Seith
+Seidler
+Seehusen
+Seefried
+Seed
+Scovell
+Scorzelli
+Sconiers
+Schwind
+Schwichtenber
+Schwerin
+Schwenke
+Schwaderer
+Schussler
+Schuneman
+Schumpert
+Schultheiss
+Schroll
+Schroepfer
+Schroeden
+Schrimpf
+Schook
+Schoof
+Schomburg
+Schoenfeldt
+Schoener
+Schnoor
+Schmick
+Schlereth
+Schindele
+Schildt
+Schildknecht
+Schemmel
+Scharfenberg
+Schanno
+Schane
+Schaer
+Schad
+Scearce
+Scardino
+Sawka
+Sawinski
+Savoca
+Savery
+Saults
+Saucer
+Sarpy
+Saris
+Sardinha
+Sarafin
+Sankar
+Sanjurjo
+Sanderfer
+Sanagustin
+Samudio
+Sammartino
+Samas
+Salz
+Salmen
+Sallie
+Salkeld
+Salamon
+Sakurai
+Sakoda
+Safley
+Sada
+Sachse
+Ryden
+Ryback
+Russow
+Russey
+Ruprecht
+Rumple
+Ruffini
+Rudzinski
+Rudel
+Rudden
+Rud
+Rovero
+Routledge
+Roussin
+Rousse
+Rouser
+Rougeau
+Rosie
+Rosica
+Romey
+Romaniello
+Rolfs
+Rogoff
+Rogne
+Rodriquz
+Rodrequez
+Rodin
+Rocray
+Rocke
+Robbin
+Riviere
+Rivette
+Riske
+Risenhoover
+Rindfleisch
+Rinaudo
+Rimbey
+Riha
+Righi
+Ridner
+Ridling
+Riden
+Rhue
+Reyome
+Reynoldson
+Reusch
+Rensing
+Rensch
+Rennels
+Renderos
+Reininger
+Reiners
+Reigel
+Rehmer
+Regier
+Reff
+Reef
+Redlin
+Recchia
+Reaume
+Reagor
+Rayne
+Rawe
+Rattigan
+Raska
+Rashed
+Ranta
+Ranft
+Randlett
+Randa
+Ramiez
+Ramella
+Rallis
+Rajan
+Raisbeck
+Raimondo
+Raible
+Ragone
+Rackliffe
+Quirino
+Quiring
+Quero
+Quaife
+Pyke
+Purugganan
+Pursifull
+Purkett
+Purdon
+Punches
+Pun
+Pulos
+Pulling
+Puccia
+Provance
+Propper
+Preis
+Prehn
+Prata
+Prasek
+Pranger
+Pradier
+Portor
+Portley
+Porte
+Popiel
+Popescu
+Pomales
+Polowy
+Pollett
+Politis
+Polit
+Poley
+Pol
+Pohler
+Poggio
+Poet
+Podolak
+Poag
+Plymel
+Ploeger
+Planty
+Piskura
+Pirrone
+Pirro
+Piroso
+Pinsky
+Pile
+Pilant
+Pickerill
+Piccolomini
+Picart
+Piascik
+Phann
+Petruzzelli
+Petosa
+Persson
+Perretta
+Perkowski
+Perilli
+Percifield
+Perault
+Peppel
+Pember
+Pelotte
+Pelcher
+Peixoto
+Pehl
+Peatross
+Pearlstein
+Peacher
+Payden
+Paya
+Pawelek
+Pavey
+Pauda
+Pathak
+Parrillo
+Parness
+Parlee
+Paoli
+Pannebaker
+Palomar
+Palo
+Palmberg
+Paganelli
+Paffrath
+Padovano
+Padden
+Pachucki
+Over
+Ovando
+Othman
+Osowski
+Osler
+Osika
+Orsburn
+Orlowsky
+Oregel
+Oppelt
+Opfer
+Opdyke
+Onell
+Omer
+Olivos
+Okumura
+Okoro
+Ogas
+Offer
+Oelschlaeger
+Odette
+Oder
+Ocanas
+Obrion
+Obarr
+Oas
+Oare
+Nyhus
+Nyenhuis
+Nunnelley
+Nunamaker
+Nuckels
+Noyd
+Nowlan
+Novakovich
+Noteboom
+Norviel
+Nortz
+Norment
+Norland
+Nolt
+Nolie
+Nixson
+Nitka
+Nissley
+Nishiyama
+Niland
+Niewiadomski
+Niemeier
+Nieland
+Nickey
+Nicholsen
+Newark
+Neugent
+Neto
+Nerren
+Nein
+Neikirk
+Neigh
+Nedrow
+Neave
+Nazaire
+Navaro
+Navalta
+Nasworthy
+Nasif
+Nani
+Nalepa
+Nakao
+Nakai
+Nadolny
+Myklebust
+Mussel
+Murthy
+Muratore
+Murat
+Mundie
+Mulverhill
+Muilenburg
+Muetzel
+Mudra
+Mudgett
+Mrozinski
+Moura
+Mottinger
+Morson
+Moretto
+Morentin
+Mordan
+Mooreland
+Mooers
+Monts
+Montone
+Montondo
+Montiero
+Monserrate
+Monie
+Monat
+Monares
+Mollo
+Mollet
+Molacek
+Mokry
+Mohrmann
+Mohabir
+Mogavero
+Moes
+Moceri
+Miyoshi
+Mitzner
+Misra
+Mis
+Mirr
+Mira
+Minish
+Minge
+Minckler
+Milroy
+Mille
+Mileski
+Milanesi
+Miko
+Mihok
+Mihalik
+Mieczkowski
+Messerli
+Meskill
+Mesenbrink
+Merton
+Merryweather
+Merkl
+Menser
+Menner
+Menk
+Menden
+Menapace
+Melbourne
+Mekus
+Meinzer
+Mein
+Meers
+Mctigue
+Mcquitty
+Mcpheron
+Mcmurdie
+Mcleary
+Mclafferty
+Mckinzy
+Mckibbin
+Mckethan
+Mcintee
+Mcgurl
+Mceachran
+Mcdowall
+Mcdermitt
+Mccuaig
+Mccreedy
+Mccoskey
+Mcclosky
+Mcclintick
+Mccleese
+Mccanless
+Mazzucco
+Mazzocco
+Mazurkiewicz
+Mazariego
+Mayhorn
+Maxcy
+Mavity
+Mauzey
+Maulding
+Matuszewski
+Mattsson
+Mattke
+Matsushita
+Matsuno
+Matsko
+Matkin
+Mathur
+Mates
+Masterman
+Massett
+Massart
+Massari
+Mashni
+Martella
+Marren
+Margotta
+Marder
+Marczak
+Maran
+Maradiaga
+Manwarren
+Mantini
+Manter
+Mantelli
+Manso
+Mangone
+Manfredonia
+Malden
+Malboeuf
+Malanga
+Makara
+Maison
+Maisano
+Mairs
+Mailhiot
+Magri
+Magic
+Madron
+Madole
+Mackall
+Macduff
+Macartney
+Lynds
+Lusane
+Luffman
+Lua
+Louth
+Loughmiller
+Lougheed
+Lotspeich
+Lorenzi
+Loree
+Loosli
+Looker
+Longe
+Longanecker
+Lonero
+Lohmeyer
+Loeza
+Lobstein
+Lobner
+Lober
+Littman
+Litalien
+Lippe
+Lints
+Linear
+Lijewski
+Ligas
+Liebert
+Liebermann
+Liberati
+Lezcano
+Levinthal
+Lessor
+Less
+Lesieur
+Lenning
+Lengel
+Len
+Lempke
+Lemp
+Lemar
+Leitzke
+Leinweber
+Legrone
+Lege
+Leder
+Lawnicki
+Lauth
+Laun
+Laughary
+Latin
+Lassley
+Lashway
+Larrivee
+Largen
+Lare
+Lanouette
+Lanno
+Langille
+Langen
+Landing
+Lana
+Lamonte
+Lalin
+Lala
+Laible
+Lafratta
+Laforte
+Lacuesta
+Lacer
+Labore
+Laboe
+Labeau
+Kwasniewski
+Kunselman
+Kuhr
+Kuchler
+Kuc
+Krugman
+Kruckenberg
+Krotzer
+Kroemer
+Krist
+Krigbaum
+Kreke
+Kreisman
+Kreisler
+Kreft
+Krasnow
+Kras
+Krag
+Kouyate
+Kough
+Kotz
+Kostura
+Korner
+Kornblum
+Korczynski
+Koppa
+Kopczyk
+Konz
+Komorowski
+Kollen
+Kolander
+Koepnick
+Koehne
+Kochis
+Knoch
+Knippers
+Knaebel
+Klipp
+Klinedinst
+Klimczyk
+Klier
+Klement
+Klaphake
+Kisler
+Kinzie
+Kines
+Kindley
+Kimple
+Kimm
+Kimbel
+Kilker
+Kilborn
+Kibbey
+Khong
+Ketchie
+Kerbow
+Kennemore
+Kennebeck
+Kenneally
+Kenndy
+Kenmore
+Kemnitz
+Kemler
+Kemery
+Kelnhofer
+Kellstrom
+Kellis
+Kellams
+Keiter
+Keirstead
+Keeny
+Keelin
+Keefauver
+Keams
+Kautzman
+Kaus
+Katayama
+Kasson
+Kassim
+Kasparian
+Kase
+Karwoski
+Kapuscinski
+Kaneko
+Kamerling
+Kamada
+Kalka
+Kalar
+Kakacek
+Kaczmarczyk
+Jurica
+Junes
+Journell
+Jolliffe
+Johnsey
+Joel
+Jindra
+Jimenz
+Jette
+Jesperson
+Jerido
+Jenrette
+Jencks
+Jech
+Jayroe
+Jayo
+Jaye
+Javens
+Jaskot
+Jaros
+Jaquet
+Janowiak
+Jame
+Jaegers
+Jackel
+Izumi
+Ith
+Italia
+Irelan
+Ion
+Inzunza
+Imoto
+Imme
+Iglehart
+Iannone
+Iannacone
+Huyler
+Hussaini
+Hurlock
+Hurlbutt
+Huprich
+Humphry
+Hulslander
+Huelsman
+Hudelson
+Hudecek
+Hsia
+Hreha
+Hoyland
+Howk
+Housholder
+Housden
+Houff
+Horkey
+Honan
+Homme
+Holtzberg
+Hollyfield
+Hollings
+Hollenbaugh
+Hokenson
+Hogrefe
+Hogland
+Hoel
+Hodgkin
+Hochhalter
+Hjelle
+Hittson
+Hinderman
+Hinchliffe
+Hime
+Hilyer
+Hilby
+Hibshman
+Heydt
+Hewell
+Heward
+Hetu
+Hestand
+Heslep
+Herridge
+Herner
+Hernande
+Hermandez
+Hermance
+Herbold
+Heon
+Henthorne
+Henion
+Henao
+Heming
+Helmkamp
+Hellberg
+Heidgerken
+Heichel
+Hehl
+Hegedus
+Hefty
+Heckathorne
+Hearron
+Haymer
+Haycook
+Havlicek
+Hausladen
+Haseman
+Hartsook
+Hartog
+Harns
+Harne
+Harmann
+Haren
+Hanserd
+Hanners
+Hanekamp
+Hamra
+Hamley
+Hamelin
+Hamblet
+Hakimi
+Hagle
+Hagin
+Haehn
+Haeck
+Hackleman
+Haacke
+Gulan
+Guirand
+Guiles
+Guggemos
+Guerrieri
+Guerreiro
+Guereca
+Gudiel
+Guccione
+Gubler
+Gruenwald
+Gritz
+Grieser
+Grewe
+Grenon
+Gregersen
+Grefe
+Greener
+Grech
+Grecco
+Gravette
+Grassia
+Granholm
+Graner
+Grandi
+Grahan
+Gradowski
+Gradney
+Graczyk
+Gouthier
+Gottschall
+Goracke
+Gootee
+Goodknight
+Goodine
+Gonzalea
+Gonterman
+Gonalez
+Gomm
+Goleman
+Goldtooth
+Goldstone
+Goldey
+Golan
+Goes
+Goen
+Goeller
+Goel
+Goecke
+Godek
+Goan
+Glunz
+Gloyd
+Glodowski
+Glinski
+Glawe
+Girod
+Girdley
+Giovanni
+Gindi
+Gillings
+Gildner
+Giger
+Giesbrecht
+Gierke
+Gier
+Giboney
+Giaquinto
+Giannakopoulo
+Giaimo
+Giaccio
+Giacalone
+Gessel
+Gerould
+Gerlt
+Gerhold
+Geralds
+Genson
+Genereux
+Gellatly
+Geigel
+Gehrig
+Gehle
+Geerdes
+Geagan
+Gawel
+Gavina
+Gauss
+Gatwood
+Gathman
+Gaster
+Garske
+Garratt
+Garms
+Garis
+Gansburg
+Gammell
+Gambale
+Gamba
+Galimore
+Gadway
+Gadoury
+Furrer
+Furnish
+Furino
+Fullard
+Fukui
+Fuhrer
+Fryou
+Friesner
+Friedli
+Friedl
+Friedberg
+Freyermuth
+Fremin
+Fredell
+Fraze
+Franken
+Fought
+Foth
+Fote
+Fortini
+Fornea
+Formanek
+Forker
+Forgette
+Folan
+Foister
+Foglesong
+Flinck
+Flewellen
+Flaten
+Flaig
+Fitgerald
+Fischels
+Firman
+Finstad
+Finkelman
+Finister
+Finder
+Fina
+Fettes
+Fetterhoff
+Ferriter
+Ferch
+Fennessy
+Feltus
+Feltes
+Feinman
+Farve
+Farry
+Farrall
+Farag
+Falzarano
+Falck
+Falanga
+Fakhoury
+Faire
+Fairbrother
+Fagley
+Faggins
+Facteau
+Ewer
+Ewbank
+Evola
+Evener
+Eustis
+Eugenio
+Estwick
+Estel
+Essa
+Espinola
+Escutia
+Eschmann
+Erpelding
+Ernsberger
+Erling
+Entz
+Enrique
+Engelhart
+Enbody
+Emick
+Elsinger
+Ellinwood
+Ellingsen
+Ellicott
+Elkind
+Eisinger
+Eisenbeisz
+Eischen
+Eimer
+Eigner
+Eichhorst
+Ehmke
+Egleston
+Eggett
+Ege
+Efurd
+Edgeworth
+Eckels
+Ebey
+Eberling
+Eagleton
+Dwiggins
+Dweck
+Dunnings
+Dunnavant
+Dumler
+Duman
+Dugue
+Duerksen
+Dudeck
+Dreisbach
+Drawdy
+Drawbaugh
+Draine
+Draggoo
+Dowse
+Dovel
+Doughton
+Douds
+Doubrava
+Dort
+Dorshorst
+Dornier
+Doolen
+Donavan
+Dominque
+Dominion
+Dominik
+Domingez
+Dome
+Dom
+Dolder
+Dold
+Dobies
+Dk
+Diskin
+Disano
+Dirden
+Diponio
+Dipirro
+Dimock
+Diltz
+Dillabough
+Diley
+Dikes
+Digges
+Digerolamo
+Diel
+Dicker
+Dicharry
+Dicecco
+Dibartolomeo
+Diamant
+Dewire
+Devone
+Dessecker
+Dertinger
+Derousselle
+Derk
+Depauw
+Depalo
+Denherder
+Demeyer
+Demetro
+Demastus
+Delvillar
+Deloye
+Delosrios
+Delgreco
+Delarge
+Delangel
+Dejongh
+Deitsch
+Degiorgio
+Degidio
+Defreese
+Defoe
+Decambra
+Debenedetto
+Deaderick
+Daza
+Dauzat
+Daughenbaugh
+Dato
+Dass
+Darwish
+Dantuono
+Danton
+Dammeyer
+Daloia
+Daleo
+Dagg
+Dacey
+Curts
+Cuny
+Cunneen
+Culverhouse
+Cuervo
+Cucinella
+Cubit
+Crumm
+Crudo
+Crowford
+Crout
+Crotteau
+Crossfield
+Crooke
+Crom
+Critz
+Cristaldi
+Crickmore
+Cribbin
+Cremeens
+Crayne
+Cradduck
+Couvertier
+Cottam
+Cossio
+Correy
+Cordrey
+Coplon
+Copass
+Coone
+Coody
+Contois
+Consla
+Connelley
+Connard
+Congo
+Congleton
+Condry
+Conception
+Coltey
+Colindres
+Colgrove
+Colfer
+Colasurdo
+Cocker
+Cochell
+Cobbin
+Clouthier
+Closs
+Cloonan
+Clizbe
+Clennon
+Clayburn
+Claybourn
+Clausell
+Clasby
+Clagett
+Ciskowski
+Cirrincione
+Cinque
+Cinelli
+Cimaglia
+Ciaburri
+Christiani
+Christeson
+Chladek
+Chizmar
+Chinnici
+Chiarella
+Chevrier
+Cheves
+Chernow
+Cheong
+Chelton
+Charlette
+Chanin
+Cham
+Chaligoj
+Celestino
+Cayce
+Cavey
+Cavaretta
+Caughron
+Catmull
+Catapano
+Casio
+Cashaw
+Carullo
+Carualho
+Carthon
+Cartelli
+Carruba
+Carrere
+Carolus
+Carmine
+Carlstrom
+Carli
+Carfora
+Carello
+Carbary
+Car
+Caplette
+Cannell
+Cancilla
+Campell
+Cammarota
+Camilo
+Camejo
+Camarata
+Caisse
+Cacioppo
+Cabbagestalk
+Cabatu
+Cabanas
+Byles
+Buxbaum
+Butland
+Butch
+Burrington
+Burnsed
+Burningham
+Burlingham
+Burgy
+Buitrago
+Buffett
+Bueti
+Buehring
+Buday
+Bucks
+Bucknell
+Buchbinder
+Bucey
+Bruster
+Brunston
+Brumby
+Bruins
+Brouillet
+Brosious
+Broomes
+Brodin
+Broddy
+Brochard
+Britsch
+Britcher
+Brierley
+Brezina
+Bressi
+Bressette
+Breslow
+Brenden
+Breier
+Brei
+Braymer
+Brasuell
+Brash
+Branscomb
+Branin
+Brandley
+Brahler
+Bracht
+Bracamontes
+Brabson
+Boyne
+Boxell
+Bowery
+Bovard
+Boutelle
+Boulette
+Bottini
+Botkins
+Bosen
+Boscia
+Boscarino
+Borich
+Bores
+Boreman
+Bordoy
+Bordley
+Bordenet
+Boquet
+Boocks
+Bolner
+Boissy
+Boilard
+Bohnen
+Bohall
+Boening
+Boccia
+Boccella
+Bobe
+Blyth
+Blitz
+Blew
+Blacksmith
+Biviano
+Bitto
+Bisel
+Binstock
+Bines
+Billiter
+Bigsby
+Bighorse
+Bielawski
+Bickmore
+Bettin
+Bettenhausen
+Besson
+Beseau
+Berton
+Berroa
+Berntson
+Bernas
+Berisford
+Berhow
+Bergsma
+Benyo
+Benyard
+Bente
+Bennion
+Benko
+Belsky
+Bellavance
+Belasco
+Belardo
+Beidler
+Behring
+Begnaud
+Bega
+Befort
+Beek
+Bedore
+Beddard
+Becknell
+Beardslee
+Beardall
+Beagan
+Bayly
+Bauza
+Bautz
+Bausman
+Baumler
+Batterson
+Battenfield
+Bassford
+Basse
+Basemore
+Baruch
+Bartholf
+Bars
+Barman
+Baray
+Barabas
+Banghart
+Banez
+Balsam
+Ballester
+Ballagh
+Baldock
+Bagnoli
+Bagheri
+Bacus
+Bacho
+Baccam
+Axson
+Averhart
+Aver
+Ave
+Austill
+Auberry
+Athans
+Atcitty
+Atay
+Astarita
+Ascolese
+Artzer
+Arts
+Arrasmith
+Argenbright
+Aresco
+Arb
+Aranjo
+Appleyard
+Appenzeller
+App
+Apilado
+Antonetti
+Antis
+Annett
+Annas
+Angwin
+Andris
+Andries
+Andreozzi
+Ando
+Andis
+Anderegg
+Anastasia
+Amyot
+Aminov
+Amelung
+Amelio
+Amason
+Alviar
+Allendorf
+Allday
+Alice
+Aldredge
+Alcivar
+Alaya
+Alapai
+Airington
+Aina
+Ailor
+Ahrns
+Ahmadi
+Agresta
+Agent
+Affolter
+Aeschlimann
+Adney
+Aderhold
+Adell
+Adachi
+Ackiss
+Aben
+Abdelhamid
+Abar
+Aase
+Zorilla
+Zordan
+Zollman
+Zoch
+Zipfel
+Zimmerle
+Zike
+Ziel
+Zhong
+Zens
+Zelada
+Zaman
+Zahner
+Zadora
+Zachar
+Zaborowski
+Zabinski
+Yzquierdo
+Yoshizawa
+Yori
+Yielding
+Yerton
+Yehl
+Yeargain
+Yeakley
+Yamaoka
+Yagle
+Yablonski
+Wynia
+Wyne
+Wyers
+Wrzesinski
+Wrye
+Wriston
+Woolums
+Woolen
+Woodlock
+Woodle
+Wonser
+Wombacher
+Wollschlager
+Wollen
+Wolfley
+Wolfer
+Wisse
+Wisell
+Wirsing
+Winstanley
+Winsley
+Winiecki
+Winiarski
+Winge
+Winesett
+Windell
+Winberry
+Willyard
+Willemsen
+Wilkosz
+Wilensky
+Wikle
+Wiford
+Wienke
+Wieneke
+Wiederhold
+Wiebold
+Widick
+Wickenhauser
+Whitrock
+Whisner
+Whinery
+Wherley
+Whedbee
+Wheadon
+Whary
+Wessling
+Wessells
+Wenninger
+Wendroth
+Wende
+Wellard
+Weirick
+Weinkauf
+Wehrman
+Weech
+Weathersbee
+Waterford
+Warton
+Warncke
+Warm
+Wardrip
+Walstrom
+Walks
+Walkowski
+Walcutt
+Waight
+Wai
+Wagman
+Waggett
+Wadford
+Vowles
+Vormwald
+Vondran
+Vohs
+Vitt
+Vitalo
+Viser
+Vinas
+Villena
+Villaneuva
+Villafranca
+Villaflor
+Vilain
+Vigilante
+Vicory
+Viana
+Vian
+Vial
+Verucchi
+Verra
+Venzke
+Venske
+Veley
+Veile
+Veeder
+Vaske
+Vasconez
+Vargason
+Varble
+Vanwert
+Vantol
+Vanscooter
+Vanmetre
+Vanmaanen
+Vanhise
+Vanetta
+Vaneaton
+Vandyk
+Vandriel
+Vandorp
+Vandewater
+Vandervelden
+Vanderstelt
+Vanderhoef
+Vanderbeck
+Vanbibber
+Vanalstine
+Vanacore
+Valdespino
+Vaill
+Vailes
+Vagliardo
+Ursini
+Urrea
+Urive
+Uriegas
+Umphress
+Ucci
+Uballe
+Tyrone
+Tynon
+Twiner
+Tutton
+Tudela
+Tuazon
+Troisi
+Tripplett
+Trias
+Trescott
+Treichel
+Tredo
+Tranter
+Tozer
+Toxey
+Tortorici
+Tornow
+Topolski
+Topia
+Topel
+Topalian
+Tonne
+Tondre
+Tola
+Toepke
+Tiu
+Tisdell
+Tiscareno
+Thornborrow
+Thomison
+Thilges
+Theuret
+Therien
+Thang
+Thagard
+Thacher
+Texter
+Terzo
+Teresa
+Tep
+Tenpenny
+Tempesta
+Teetz
+Teaff
+Tavella
+Taussig
+Tatton
+Tasler
+Tarrence
+Tardie
+Tarazon
+Tantillo
+Tanney
+Tankson
+Tangen
+Tamburo
+Takes
+Tabone
+Szilagyi
+Syphers
+Swistak
+Swiatkowski
+Sweigert
+Swayzer
+Swapp
+Svehla
+Sutphen
+Sutch
+Susa
+Surma
+Surls
+Sundermeyer
+Sundeen
+Sulek
+Suite
+Sughrue
+Sudol
+Sturms
+Stupar
+Stum
+Stuckman
+Strole
+Strohman
+Streed
+Strebeck
+Strausser
+Strassel
+Stpaul
+Storts
+Storr
+Stommes
+Stmary
+Stjulien
+Stika
+Stiggers
+Sthill
+Stevick
+Sterman
+Stephany
+Stepanek
+Stemler
+Stelman
+Stelmack
+Steinkamp
+Steinbock
+Stcroix
+Stcharles
+Staudinger
+Starry
+Stanly
+Stallsworth
+Stalley
+Stains
+Srock
+Spritzer
+Spracklin
+Spinuzzi
+Spidell
+Spice
+Speyrer
+Sperbeck
+Spendlove
+Speedy
+Speckman
+Spargur
+Spangenberg
+Spaid
+Sowle
+Soulier
+Sotolongo
+Sostre
+Sorey
+Sonier
+Somogyi
+Somera
+Solo
+Soldo
+Sofia
+Soderholm
+Snoots
+Snooks
+Snoke
+Snodderly
+Snide
+Snee
+Smoke
+Smithhart
+Smillie
+Smay
+Smallman
+Sliwinski
+Slentz
+Sledd
+Slager
+Skogen
+Skog
+Skarda
+Skalicky
+Siwek
+Sitterson
+Sisti
+Sissel
+Sis
+Sinopoli
+Similton
+Simila
+Simenson
+Silvertooth
+Silos
+Siggins
+Sieler
+Siburt
+Sianez
+Shurley
+Shular
+Shuecraft
+Shreeves
+Shon
+Shollenberger
+Shoen
+Shishido
+Shipps
+Shipes
+Shinall
+Sherfield
+Shawe
+Sharrett
+Sharrard
+Shankman
+Shan
+Sham
+Sessum
+Serviss
+Servello
+Serice
+Serda
+Semler
+Semenza
+Selmon
+Sellen
+Seley
+Seidner
+Seib
+Sehgal
+Seelbach
+Sedivy
+Sebren
+Sebo
+Seanez
+Seagroves
+Seagren
+Seagrave
+Seabron
+Schwertner
+Schwegel
+Schwarzer
+Schrunk
+Schriefer
+Schreder
+Schrank
+Schopp
+Schonfeld
+Schoenwetter
+Schnall
+Schnackenberg
+Schnack
+Schmutzler
+Schmierer
+Schmidgall
+Schlup
+Schloemer
+Schlitt
+Schermann
+Scherff
+Schellenberg
+Schain
+Schaedler
+Schabel
+Scaccia
+Saye
+Saxman
+Saurez
+Sasseen
+Sasnett
+Sas
+Sarti
+Sarra
+Sarber
+Saran
+Santoy
+Santeramo
+Sansoucy
+Sando
+Sandles
+Sandburg
+Sandau
+Samra
+Samaha
+Salon
+Salizar
+Salam
+Saindon
+Sagaser
+Saeteun
+Sadusky
+Sackman
+Sabater
+Saas
+Ruthven
+Ruszkowski
+Rusche
+Rumpf
+Ruhter
+Ruhenkamp
+Rufo
+Rudge
+Ruddle
+Rowlee
+Rowand
+Routhier
+Rougeot
+Rotramel
+Rotan
+Roswell
+Rosten
+Rosillo
+Rookard
+Roode
+Rongstad
+Rollie
+Roider
+Roffe
+Roettger
+Rodick
+Rochez
+Rochat
+Roads
+Rivkin
+Rivadeneira
+Riston
+Risso
+Rise
+Rinderknecht
+Riis
+Riggsbee
+Rifkin
+Rieker
+Riegle
+Riedy
+Richwine
+Richmon
+Ricciuti
+Riccardo
+Ricardson
+Rhew
+Revoir
+Revier
+Remsberg
+Remiszewski
+Rembold
+Rella
+Reinken
+Reiland
+Reidel
+Reichart
+Rehak
+Redway
+Rednour
+Redifer
+Redgate
+Redenbaugh
+Redburn
+Reap
+Readus
+Raybuck
+Rauhuff
+Rauda
+Ratte
+Rathje
+Rappley
+Rands
+Ramseyer
+Ramseur
+Ramsdale
+Ramo
+Ramariz
+Raitz
+Raisch
+Rainone
+Rahr
+Ragasa
+Rafalski
+Radunz
+Quenzer
+Queja
+Queenan
+Pyun
+Puz
+Putzier
+Puskas
+Purrington
+Puri
+Punt
+Pullar
+Pruse
+Pring
+Primeau
+Prevette
+Preuett
+Presto
+Prestage
+Pownell
+Pownall
+Potthoff
+Potratz
+Poth
+Poter
+Posthuma
+Posen
+Porritt
+Popkin
+Poormon
+Polidoro
+Poles
+Polcyn
+Pokora
+Poer
+Pluviose
+Plock
+Pleva
+Placke
+Pioli
+Pingleton
+Pinchback
+Pinch
+Pieretti
+Piccone
+Piatkowski
+Philley
+Phibbs
+Phay
+Phagan
+Pfund
+Peyer
+Pettersen
+Petter
+Petrucelli
+Petropoulos
+Petras
+Petix
+Pester
+Perks
+Pepperman
+Pennick
+Penado
+Pelot
+Pelis
+Peeden
+Pechon
+Peal
+Pazmino
+Patchin
+Pasierb
+Parran
+Parilla
+Pardy
+Parcells
+Paragas
+Paradee
+Papin
+Panko
+Pangrazio
+Pangelinan
+Pandya
+Pancheri
+Panas
+Palmiter
+Pallares
+Palinkas
+Palek
+Pagliaro
+Packham
+Pacitti
+Ozier
+Overbaugh
+Oursler
+Ouimette
+Otteson
+Otsuka
+Othon
+Osmundson
+Oroz
+Orgill
+Ordeneaux
+Orama
+Oppy
+Opheim
+Onkst
+Oltmanns
+Olstad
+Olofson
+Ollivier
+Olen
+Olejniczak
+Okura
+Okuna
+Okey
+Ohrt
+Oharra
+Oguendo
+Ogier
+Offermann
+Oetzel
+Oechsle
+Odor
+Odoherty
+Oddi
+Ockerman
+Occhiogrosso
+Obryon
+Obremski
+Nyreen
+Nylund
+Nylen
+Nyholm
+Nuon
+Nuanes
+Norrick
+Noris
+Nordell
+Norbury
+Nooner
+Nono
+Nomura
+Nole
+Nolden
+Nola
+Nofsinger
+Nocito
+Nobel
+Niedbala
+Niebergall
+Nicolini
+Nicole
+Nicklaus
+Nevils
+Neuburger
+Nemerofsky
+Nemecek
+Nazareno
+Nastri
+Nast
+Nancy
+Nagorski
+Myre
+Muzzey
+Mutton
+Mutschler
+Muther
+Musumeci
+Muranaka
+Muramoto
+Murad
+Murach
+Muns
+Munno
+Muncrief
+Mugrage
+Muecke
+Mozer
+Moyet
+Mowles
+Mottern
+Mosman
+Mosconi
+Morine
+Morge
+Moravec
+Morad
+Moneymaker
+Mones
+Moncur
+Monarez
+Molzahn
+Moglia
+Moesch
+Mody
+Modisett
+Mitnick
+Mithcell
+Mitchiner
+Mistry
+Misercola
+Mirabile
+Minvielle
+Mino
+Minkler
+Minifield
+Minichiello
+Mindell
+Minasian
+Milteer
+Millwee
+Millstein
+Millien
+Mikrut
+Mihaly
+Miggins
+Michard
+Mezo
+Metzner
+Mesquita
+Mervin
+Merriwether
+Merk
+Merfeld
+Mercik
+Mercadante
+Mention
+Menna
+Mendizabal
+Mender
+Members
+Melusky
+Melquist
+Mellado
+Meler
+Melendes
+Mekeel
+Meiggs
+Megginson
+Meck
+Mcwherter
+Mcwayne
+Mcsparren
+Mcrea
+Mcneff
+Mcnease
+Mcmurrin
+Mckeag
+Mchughes
+Mcguiness
+Mcgilton
+Mcelreath
+Mcelhone
+Mcelhenney
+Mceldowney
+Mccurtain
+Mccure
+Mccosker
+Mccory
+Mccormic
+Mccline
+Mccleave
+Mcclatchey
+Mccarney
+Mccanse
+Mcallen
+Mazzie
+Mazin
+Mazanec
+Mayette
+Mautz
+Mauser
+Maun
+Mattas
+Mathurin
+Mathiesen
+Massmann
+Masri
+Masias
+Mascolo
+Mascetti
+Mascagni
+Marzolf
+Maruska
+Martain
+Marta
+Marszalek
+Marolf
+Marmas
+Marlor
+Markwood
+Marines
+Marinero
+Marier
+Marich
+Marcom
+Marciante
+Marchman
+Marchio
+Marbach
+Manzone
+Mantey
+Mannina
+Manhardt
+Manfred
+Manaois
+Malmgren
+Mallonee
+Mallin
+Mallary
+Malette
+Makinson
+Makins
+Makarewicz
+Mainwaring
+Maida
+Maiava
+Magro
+Magouyrk
+Magett
+Maeder
+Madyun
+Maduena
+Maden
+Madeira
+Macnamara
+Mackins
+Mackel
+Macinnes
+Macia
+Macgowan
+Lyssy
+Lyerly
+Lyalls
+Lutter
+Lunney
+Luksa
+Ludeman
+Lucidi
+Lucci
+Lowden
+Lovier
+Loughridge
+Losch
+Lory
+Lorson
+Lorenzano
+Lorden
+Lorber
+Lopardo
+Loosier
+Loomer
+Longsdorf
+Longchamps
+Loncar
+Loker
+Logwood
+Loeffelholz
+Lockmiller
+Livoti
+Linford
+Linenberger
+Lindloff
+Lindenbaum
+Limoges
+Lilla
+Liley
+Lighthill
+Lightbourne
+Lieske
+Leza
+Levels
+Levandoski
+Leuck
+Lepere
+Leonhart
+Lenon
+Lemma
+Lemler
+Leising
+Leinonen
+Lehtinen
+Lehan
+Leetch
+Leeming
+Ledyard
+Ledwith
+Ledingham
+Leclere
+Leck
+Lebert
+Leandry
+Lazzell
+Layo
+Laye
+Laxen
+Lawther
+Lawn
+Lawerance
+Lavoy
+Lavertu
+Laverde
+Lauren
+Latouche
+Latner
+Lathen
+Last
+Laskin
+Lashbaugh
+Lascala
+Larroque
+Larick
+Laraia
+Laplume
+Lanzilotta
+Lannom
+Landrigan
+Landolt
+Landess
+Lancia
+Lamkins
+Lalla
+Lalk
+Lakeman
+Lakatos
+Laib
+Lahay
+Lagrave
+Lagerquist
+Lafoy
+Lafleche
+Lader
+Labrada
+Kwiecinski
+Kutner
+Kunshier
+Kulakowski
+Kujak
+Kuehnle
+Kubisiak
+Krzyminski
+Krugh
+Krois
+Kritikos
+Krill
+Kriener
+Krewson
+Kretzschmar
+Kretz
+Kresse
+Kreiter
+Kreischer
+Krebel
+Kraut
+Krans
+Kraling
+Krahenbuhl
+Kouns
+Kotson
+Kossow
+Kopriva
+Konkle
+Kolter
+Kolk
+Kolich
+Kohner
+Koeppen
+Koenigs
+Kock
+Kochanski
+Kobus
+Knowling
+Knouff
+Knoerzer
+Knippel
+Kloberdanz
+Kleinert
+Klarich
+Klaassen
+Kizzie
+Kisamore
+Kirn
+Kiraly
+Kipps
+Kinson
+Kinneman
+Kington
+Kine
+Kimbriel
+Kille
+Kick
+Kibodeaux
+Khamvongsa
+Keylon
+Kever
+Keser
+Kertz
+Kercheval
+Kenneth
+Kendrix
+Kendle
+Ken
+Kempt
+Kemple
+Keesey
+Keats
+Keatley
+Kazmierski
+Kazda
+Kazarian
+Kawashima
+Katsch
+Kasun
+Kassner
+Kassem
+Kasperski
+Kasinger
+Kaschak
+Karels
+Kantola
+Kana
+Kamai
+Kalthoff
+Kalla
+Kalani
+Kahrs
+Kahanek
+Kacher
+Jurasek
+Juniper
+Jungels
+Jukes
+Juelfs
+Judice
+Juda
+Ju
+Josselyn
+Jonsson
+Jonak
+Joens
+Jobson
+Jegede
+Jee
+Jeanjacques
+Jaworowski
+Jaspers
+Jannsen
+Janner
+Jankowiak
+Jank
+Janiak
+Jackowski
+Jacklin
+Jabbour
+Iyer
+Iveson
+Ivan
+Isner
+Iniquez
+Ingwerson
+Ingber
+Ina
+Imbrogno
+Ille
+Ikehara
+Iannelli
+Hyson
+Huxford
+Huseth
+Hurns
+Hurney
+Hurles
+Hunnings
+Humbarger
+Hulan
+Huisinga
+Hughett
+Hughen
+Hudler
+Hubiak
+Hricko
+How
+Hoversten
+Hottel
+Hosaka
+Horsch
+Hormann
+Hordge
+Honzell
+Homburg
+Holten
+Holme
+Hollopeter
+Hollinsworth
+Hollibaugh
+Holberg
+Hohmann
+Hoenstine
+Hodell
+Hodde
+Hobert
+Hives
+Hiter
+Hirko
+Hipolito
+Hinzmann
+Hinrichsen
+Hinger
+Hincks
+Hilz
+Hilborn
+Highley
+Higashi
+Hieatt
+Hicken
+Heverly
+Hesch
+Hervert
+Hershkowitz
+Herreras
+Hermanns
+Herget
+Henriguez
+Hennon
+Hengel
+Helmlinger
+Helmig
+Helen
+Heldman
+Heizer
+Heinitz
+Heifner
+Heidorn
+Heglin
+Heffler
+Hebner
+Heathman
+Heaslip
+Hazlip
+Haymes
+Hayase
+Hawver
+Haw
+Havermale
+Havas
+Hauber
+Hashim
+Hasenauer
+Harvel
+Hartney
+Hartel
+Harsha
+Harpine
+Harkrider
+Harkin
+Harer
+Harclerode
+Hanzely
+Hanni
+Hannagan
+Hampel
+Hammerschmidt
+Hamar
+Hallums
+Hallin
+Hainline
+Haid
+Haggart
+Hafen
+Haer
+Hadiaris
+Hadad
+Hackford
+Habeeb
+Guymon
+Guttery
+Gunnett
+Gull
+Guillette
+Guiliano
+Guilbeaux
+Guiher
+Guignard
+Guerry
+Gude
+Gucman
+Guadian
+Grzybowski
+Grzelak
+Grussendorf
+Grumet
+Gruenhagen
+Grudzinski
+Ground
+Grossmann
+Grof
+Grisso
+Grisanti
+Griffitts
+Griesbaum
+Grella
+Gregston
+Graveline
+Grandusky
+Grandinetti
+Gramm
+Goynes
+Gowing
+Goudie
+Gosman
+Gort
+Gorsline
+Goralski
+Goodstein
+Goodroe
+Goodlin
+Goodheart
+Goodhart
+Gonzelez
+Gonthier
+Goldsworthy
+Goldade
+Goettel
+Goerlitz
+Goepfert
+Goehner
+Goben
+Gobeille
+Glock
+Gliem
+Gleich
+Glasson
+Glascoe
+Gladwell
+Giusto
+Girdner
+Gipple
+Giller
+Giesing
+Giammona
+Ghormley
+Germon
+Geringer
+Gergely
+Gerberich
+Gepner
+Gens
+Genier
+Gemme
+Gelsinger
+Geigle
+Gebbia
+Gayner
+Gavitt
+Gatrell
+Gastineau
+Gasiewski
+Gascoigne
+Garro
+Garin
+Ganong
+Ganga
+Galpin
+Gallus
+Galizia
+Gajda
+Gahm
+Gagen
+Gaffigan
+Furno
+Furnia
+Furgason
+Fronczak
+Frishman
+Friess
+Frierdich
+Fresh
+Freestone
+Franta
+Frankovich
+Fors
+Forres
+Forrer
+Floris
+Florido
+Floria
+Flis
+Flicek
+Flens
+Flegal
+Flamenco
+Finkler
+Finkenbinder
+Finefrock
+Filter
+Filpo
+Filion
+Fierman
+Fieldman
+Ferreyra
+Fernendez
+Fergeson
+Fera
+Fencil
+Feith
+Feight
+Federici
+Federer
+Fechtner
+Feagan
+Fausnaugh
+Faubert
+Fata
+Farman
+Farinella
+Fantauzzi
+Fanara
+Falso
+Falardeau
+Fagnani
+Fabro
+Excell
+Ewton
+Evey
+Everetts
+Eve
+Evarts
+Etherington
+Estremera
+Estis
+Estabrooks
+Essig
+Esplin
+Espenschied
+Ernzen
+Erich
+Eppes
+Eppard
+Entwisle
+Emmi
+Emison
+Elison
+Elguezabal
+Eledge
+Elbaz
+Eisler
+Eiden
+Eichorst
+Eichert
+Egle
+Eggler
+Eggimann
+Edey
+Eckerman
+Echelberger
+Ebbs
+Ebanks
+Dziak
+Dyche
+Dyce
+Dusch
+Duross
+Durley
+Durate
+Dunsworth
+Dumke
+Dulek
+Duhl
+Duggin
+Dufford
+Dudziak
+Ducrepin
+Dubree
+Dubre
+Dubie
+Dubas
+Droste
+Drisko
+Drewniak
+Doxtator
+Dowtin
+Downum
+Doubet
+Dottle
+Dosier
+Doshi
+Dorst
+Dorset
+Dornbusch
+Doren
+Donze
+Donica
+Domanski
+Domagala
+Dohse
+Doerner
+Doerfler
+Doble
+Dobkins
+Dilts
+Digiulio
+Digaetano
+Dietzel
+Diddle
+Dickel
+Dezarn
+Devoy
+Devoss
+Devonshire
+Devon
+Devilla
+Devere
+Deters
+Desvergnes
+Deshay
+Desena
+Deross
+Der
+Depedro
+Densley
+Demorest
+Demore
+Demora
+Demirjian
+Demerchant
+Dematteis
+Demateo
+Delgardo
+Delfavero
+Delaurentis
+Delamar
+Delacy
+Deitrich
+Deisher
+Degracia
+Degraaf
+Defries
+Defilippis
+Decoursey
+Debruin
+Debiasi
+Debar
+Dearden
+Dealy
+Dayhoff
+Davino
+Darvin
+Darrisaw
+Darbyshire
+Daquino
+Daprile
+Danial
+Danh
+Danahy
+Dalsanto
+Dallavalle
+Daine
+Dagel
+Dadamo
+Dacy
+Dacunha
+Dabadie
+Czyz
+Cutsinger
+Curney
+Cuppernell
+Cunliffe
+Cumby
+Cullop
+Cullinane
+Cugini
+Cudmore
+Cuda
+Cucuzza
+Cuch
+Crumby
+Crouser
+Crock
+Critton
+Critchley
+Cristy
+Cremona
+Cremar
+Crehan
+Creary
+Crasco
+Crall
+Crabbe
+Cozzolino
+Cozier
+Coyner
+Couvillier
+Counterman
+Coulthard
+Coudriet
+Cottom
+Corzo
+Cornutt
+Corkran
+Cords
+Corda
+Copelin
+Coonan
+Consolo
+Conrow
+Conran
+Connerton
+Conkwright
+Condren
+Comp
+Comly
+Comisky
+Colli
+Collet
+Colello
+Colbeck
+Colarusso
+Coiner
+Cohron
+Codere
+Cocks
+Cobia
+Cly
+Cluster
+Clure
+Clowser
+Clovis
+Clingenpeel
+Clenney
+Clendaniel
+Clemenson
+Cleere
+Cleckler
+Claybaugh
+Clason
+Cirullo
+Ciraulo
+Ciolek
+Ciampi
+Christopherse
+Christophe
+Chovanec
+Chopra
+Chol
+Chiem
+Chestnutt
+Chesterman
+Chernoff
+Chermak
+Chelette
+Checketts
+Charpia
+Charo
+Chargois
+Champman
+Challender
+Chafins
+Cerruto
+Celi
+Cea
+Cazenave
+Cay
+Cavaluzzi
+Cauthon
+Caudy
+Catino
+Caterina
+Catano
+Castell
+Cassaro
+Cassarino
+Carrano
+Carozza
+Carow
+Carmickle
+Carlyon
+Carlew
+Cardena
+Caputi
+Capley
+Capalbo
+Canseco
+Candella
+Canal
+Campton
+Camposano
+Calleros
+Calleja
+Callegari
+Calica
+Calarco
+Calais
+Caillier
+Cahue
+Cadenhead
+Cadenas
+Cabera
+Buzzo
+Busto
+Bussmann
+Busenbark
+Burzynski
+Bursley
+Bursell
+Burle
+Burkleo
+Burkette
+Burczyk
+Bumstead
+Bullett
+Buikema
+Buenaventura
+Buege
+Buechel
+Budreau
+Budhram
+Bucknam
+Brye
+Brushwood
+Brumbalow
+Brulotte
+Bruington
+Bruderer
+Browns
+Brougher
+Bromfield
+Broege
+Brodhead
+Brocklesby
+Broadie
+Brizuela
+Britz
+Brisendine
+Brilla
+Briggeman
+Brierton
+Bridgeford
+Breyfogle
+Brevig
+Breuninger
+Bresse
+Bresette
+Brelsford
+Breitbach
+Bread
+Brayley
+Braund
+Branscom
+Brando
+Brandner
+Brahm
+Braboy
+Brabble
+Bozman
+Boyte
+Boynes
+Boyken
+Bowell
+Bowan
+Boutet
+Bouse
+Boulet
+Boule
+Bottcher
+Bosquez
+Borrell
+Boria
+Bordes
+Borchard
+Bonson
+Bonino
+Bonas
+Bonamico
+Bolstad
+Bolser
+Bollis
+Bolich
+Bolf
+Boker
+Boileau
+Bohac
+Bogucki
+Bogren
+Boeger
+Bodziony
+Bodo
+Bodley
+Boback
+Blyther
+Blight
+Blenker
+Blazina
+Blase
+Blamer
+Blacknall
+Blackmond
+Bitz
+Biser
+Biscardi
+Binz
+Bilton
+Billotte
+Billafuerte
+Bigford
+Biegler
+Bibber
+Bhandari
+Beyersdorf
+Bevelle
+Bettendorf
+Bessard
+Bertsche
+Berne
+Berlinger
+Berish
+Beranek
+Bentson
+Bentsen
+Benskin
+Benoy
+Benoist
+Benitz
+Belongia
+Belmore
+Belka
+Belen
+Beitzel
+Beiter
+Beitel
+Behrns
+Beckworth
+Becka
+Beaudion
+Beary
+Beare
+Beames
+Beabout
+Beaber
+Bazzano
+Bazinet
+Baucum
+Batrez
+Baswell
+Bastos
+Bascomb
+Bartha
+Barstad
+Barrilleaux
+Barretto
+Barresi
+Barona
+Barkhurst
+Barke
+Bardales
+Barczak
+Barca
+Barash
+Banfill
+Bambino
+Balonek
+Balmes
+Ballon
+Balko
+Balestrieri
+Baldino
+Baldelli
+Baken
+Baiza
+Bahner
+Baek
+Badour
+Badman
+Badley
+Badia
+Backmon
+Bacich
+Bacca
+Ayscue
+Ayo
+Aynes
+Austen
+Ausiello
+Auringer
+Auiles
+Aspinwall
+Askwith
+Artiga
+Arroliga
+Arns
+Arman
+Arellanes
+Aracena
+Antwine
+Antuna
+Anselmi
+Ansel
+Annen
+Angelino
+Angeli
+Angarola
+Andrae
+Amparo
+Amodio
+Amie
+Ameen
+Alwine
+Alverio
+Altro
+Altobello
+Altemus
+Alquicira
+Ally
+Allphin
+Allemand
+Allam
+Alessio
+Akpan
+Akerman
+Aiona
+Aikman
+Agyeman
+Agredano
+Adamik
+Adamczak
+Acrey
+Achilles
+Acevado
+Abu
+Abreo
+Abrahamsen
+Abild
+Zwicker
+Zweig
+Zuvich
+Zumpano
+Zuluaga
+Zubek
+Zornes
+Zoglmann
+Ziminski
+Zimbelman
+Zhanel
+Zenor
+Zechman
+Zauner
+Zamarron
+Zaffino
+Yusuf
+Ytuarte
+Yoke
+Yett
+Yerkovich
+Yelder
+Yaw
+Yasuda
+Yapp
+Yankee
+Yaden
+Yackley
+Yaccarino
+Xia
+Wytch
+Wyre
+Wussow
+Worthing
+Wormwood
+Wormack
+Worlds
+Wordsworth
+Wordell
+Woodroof
+Woodington
+Woodhams
+Wooddell
+Wollner
+Wojtkowski
+Wojcicki
+Wogan
+Wlodarczyk
+Wixted
+Withington
+Withem
+Wisler
+Wirick
+Winterhalter
+Winski
+Winne
+Winemiller
+Wimett
+Wiltfong
+Willibrand
+Willes
+Wilkos
+Wilbon
+Wiktor
+Wiggers
+Wigg
+Wiegmann
+Wickliff
+Wiberg
+Whittler
+Whittenton
+Whitling
+Whitledge
+Whitherspoon
+Whiters
+Whitecotton
+Whitebird
+Wheary
+Wetherill
+Westmark
+Westaby
+Wertenberger
+Wentland
+Wenstrom
+Wenker
+Wellen
+Weier
+Wegleitner
+Wedekind
+Wawers
+Wassel
+Warehime
+Wank
+Wandersee
+Waltmon
+Waltersheid
+Walbridge
+Wakely
+Wakeham
+Wajda
+Waithe
+Waidelich
+Wahler
+Wahington
+Wagster
+Wadel
+Vuyovich
+Vuolo
+Vulich
+Vukovich
+Volmer
+Vollrath
+Vollbrecht
+Vogelgesang
+Voeller
+Vlach
+Vivar
+Vitullo
+Vitanza
+Visker
+Visalli
+Viray
+Vinning
+Viniard
+Villapando
+Villaman
+Vier
+Viar
+Viall
+Verstraete
+Vermilya
+Verdon
+Venn
+Velten
+Velis
+Vasey
+Vanoven
+Vanorder
+Vanlue
+Vanheel
+Vanderwoude
+Vanderheide
+Vandenheuvel
+Vandenbos
+Vandeberg
+Vandal
+Vanblarcom
+Vanaken
+Vanacker
+Vallian
+Valine
+Valent
+Vaine
+Vaile
+Vadner
+Uttech
+Urioste
+Urbanik
+Unrath
+Unnasch
+Underkofler
+Uehara
+Udy
+Tyrer
+Tyburski
+Twaddle
+Turntine
+Tunis
+Tullock
+Trunk
+Tropp
+Troilo
+Tritsch
+Triola
+Trigo
+Tribou
+Tribley
+Tri
+Trethewey
+Tress
+Trela
+Treharne
+Trefethen
+Trayler
+Trax
+Traut
+Trang
+Tranel
+Trager
+Traczyk
+Towsley
+Torrecillas
+Tornatore
+Tork
+Torivio
+Toriello
+Tooles
+Toodle
+Tomme
+Tolosa
+Tolen
+Toca
+Titterington
+Tipsword
+Tinklenberg
+Tim
+Tigney
+Tigert
+Thygerson
+Thurn
+Thur
+Threats
+Thorstad
+Thornberg
+Thoresen
+Thomaston
+Tholen
+Thicke
+Theiler
+Thebeau
+Theaux
+Thaker
+Tewani
+Teufel
+Tetley
+Terrebonne
+Terrano
+Terpening
+Telly
+Tela
+Teig
+Teichert
+Tegethoff
+Teele
+Tatar
+Tashjian
+Tarte
+Tanton
+Tanimoto
+Tamimi
+Tamas
+Talman
+Taal
+Szydlowski
+Szostak
+Swoyer
+Swerdlow
+Sweeden
+Sweda
+Swanke
+Swander
+Swackhammer
+Suyama
+Suriano
+Suri
+Surdam
+Suprenant
+Sundet
+Summerton
+Sult
+Suleiman
+Suffridge
+Suby
+Stych
+Studeny
+Stubbins
+Strupp
+Struckman
+Strief
+Strictland
+Stremcha
+Strehl
+Stramel
+Stoy
+Stoutamire
+Storozuk
+Stordahl
+Stopher
+Stolley
+Stolfi
+Stoeger
+Stockhausen
+Stjulian
+Stivanson
+Stinton
+Stinchfield
+Stigler
+Stieglitz
+Stgermaine
+Steuer
+Steuber
+Steuart
+Stepter
+Stepnowski
+Stepanian
+Steimer
+Stefanelli
+Stebner
+Stears
+Steans
+Stayner
+Staubin
+Statz
+Stasik
+Starn
+Starmer
+Stargel
+Stanzione
+Stankovich
+Stan
+Stamour
+Staib
+Stadelman
+Stadel
+Stachura
+Squadrito
+Sprinkles
+Springstead
+Spragg
+Spigelmyer
+Spieler
+Spielberg
+Spaur
+Sovocool
+Sovereign
+Soundara
+Soulia
+Souffrant
+Sos
+Sorce
+Sonkin
+Sodhi
+Soble
+Sniffen
+Smouse
+Smittle
+Smithee
+Smedick
+Smaller
+Slowinski
+Slovacek
+Slominski
+Slice
+Skowronek
+Skokan
+Skanes
+Sivertson
+Sinyard
+Sinka
+Sinard
+Simonin
+Simonian
+Simmions
+Silcott
+Silberg
+Siefken
+Siddon
+Shuttlesworth
+Shubin
+Shubeck
+Shiro
+Shiraki
+Shipper
+Shina
+Shilt
+Shikles
+Shideler
+Shenton
+Shelvey
+Shellito
+Shelhorse
+Shawcroft
+Shatto
+Shanholtzer
+Shamonsky
+Shall
+Shadden
+Seymer
+Seyfarth
+Sewer
+Setlock
+Servant
+Serratos
+Serr
+Sepulueda
+Senay
+Semmel
+Semans
+Selvig
+Selkirk
+Selk
+Seligson
+Seldin
+Seiple
+Seiersen
+Seidling
+Seidensticker
+Secker
+Searson
+Scordo
+Scollard
+Scoggan
+Scobee
+Sciandra
+Scialdone
+Schwimmer
+Schwieger
+Schweer
+Schwanz
+Schutzenhofer
+Schuetze
+Schrodt
+Schriever
+Schriber
+Schremp
+Schrecongost
+Schraeder
+Schonberg
+Scholtz
+Scholle
+Schoettle
+Schoenemann
+Schoene
+Schnitker
+Schmuhl
+Schmith
+Schlotterbeck
+Schleppenbach
+Schlee
+Schickel
+Schibi
+Schein
+Scheide
+Scheibe
+Scheib
+Schaumberg
+Schardein
+Schaalma
+Scantlin
+Scantlebury
+Sayle
+Sausedo
+Saurer
+Sassone
+Sarracino
+Saric
+Sanz
+Santino
+Santarpia
+Santano
+Santaniello
+Sangha
+Sandvik
+Sandoral
+Sandobal
+Sandercock
+Sanantonio
+Salviejo
+Salsberry
+Salois
+Salazer
+Sagon
+Saglibene
+Sagel
+Sagal
+Saetern
+Saefong
+Sadiq
+Sabori
+Saballos
+Rygiel
+Rushlow
+Runco
+Rulli
+Ruller
+Ruffcorn
+Ruess
+Ruebush
+Rudlong
+Rudin
+Rudgers
+Rudesill
+Ruderman
+Rucki
+Rucinski
+Rubner
+Rubinson
+Rubiano
+Ruan
+Roznowski
+Rozanski
+Rowson
+Rower
+Rounsaville
+Roudabush
+Rotundo
+Rothell
+Rotchford
+Rosiles
+Roshak
+Rosetti
+Rosenkranz
+Rorer
+Rollyson
+Rokosz
+Rojek
+Roitman
+Rohrs
+Rogel
+Roewe
+Rodriges
+Rodocker
+Rodgerson
+Rodan
+Rodak
+Rocque
+Rochholz
+Rochel
+Robicheau
+Robbinson
+Roady
+Ritchotte
+Ripplinger
+Rippetoe
+Ringstaff
+Ringenberg
+Rinard
+Rigler
+Rightmire
+Riesen
+Riek
+Ridges
+Richner
+Richberg
+Riback
+Rial
+Rhyner
+Rhees
+Resse
+Renno
+Renee
+Rendleman
+Ren
+Reisz
+Reisenauer
+Reinschmidt
+Reins
+Reinholt
+Reinard
+Reifsnyder
+Rehfeld
+Reha
+Regester
+Reffitt
+Redler
+Rediske
+Reckner
+Reckart
+Rebolloso
+Rebollar
+Reasonover
+Reasner
+Reaser
+Reano
+Reagh
+Raval
+Ratterman
+Ratigan
+Rater
+Rasp
+Raneses
+Randolf
+Ramil
+Ramdas
+Ramberg
+Rajaniemi
+Rail
+Raid
+Raggio
+Ragel
+Ragain
+Rade
+Radaker
+Racioppi
+Rabinovich
+Quickle
+Quertermous
+Queal
+Quartucci
+Quander
+Quain
+Pynes
+Putzel
+Purl
+Pulizzi
+Pugliares
+Prusak
+Prueter
+Protano
+Propps
+Primack
+Prieur
+Presta
+Preister
+Prawl
+Pratley
+Prairie
+Pozzo
+Powless
+Povey
+Pottorf
+Pote
+Postley
+Porzio
+Ports
+Portney
+Ponzi
+Pontoriero
+Ponto
+Pont
+Poncedeleon
+Polimeni
+Polhamus
+Pole
+Polan
+Poetker
+Poellnitz
+Podgurski
+Plotts
+Pliego
+Plaugher
+Plantenberg
+Plair
+Plagmann
+Pizzitola
+Pittinger
+Pitcavage
+Pischke
+Piontek
+Pintar
+Pinnow
+Pinneo
+Pinley
+Pingel
+Pinello
+Pimenta
+Pillard
+Piker
+Pietras
+Piere
+Picasso
+Phillps
+Pfleger
+Pfahl
+Pezzuti
+Petruccelli
+Petrello
+Peteet
+Pescatore
+Peruzzi
+Perusse
+Perotta
+Perona
+Perini
+Peretti
+Perelman
+Perciful
+Peppin
+Pennix
+Pennino
+Penalosa
+Pemble
+Pelz
+Peltzer
+Pelphrey
+Pelote
+Pellum
+Pellecchia
+Pelikan
+Peitz
+Peels
+Pebworth
+Peary
+Pawlicki
+Pavelich
+Paster
+Pasquarella
+Paskey
+Paseur
+Paschel
+Parslow
+Parrow
+Parrot
+Parlow
+Parlett
+Parler
+Pargo
+Parco
+Paprocki
+Panepinto
+Panebianco
+Pandy
+Pandey
+Pamphile
+Pamintuan
+Pamer
+Paluso
+Paleo
+Paker
+Pagett
+Paczkowski
+Ozburn
+Ovington
+Overmeyer
+Ouellet
+Osterlund
+Oslin
+Oseguera
+Osaki
+Orrock
+Ormsbee
+Orlikowski
+Organista
+Oregan
+Orebaugh
+Orabuena
+Openshaw
+Ontiveroz
+Ondo
+Omohundro
+Ollom
+Ollivierre
+Olivencia
+Oley
+Olazabal
+Okino
+Oki
+Offenberger
+Oestmann
+Ocker
+Obar
+Oakeson
+Nuzum
+Nurre
+Nowinski
+Novosel
+Norquist
+Nordlie
+Noorani
+Nonnemacher
+Nolder
+Njoku
+Niznik
+Niwa
+Niss
+Ninneman
+Niner
+Nimtz
+Niemczyk
+Nieder
+Nicolo
+Nichlos
+Niblack
+Newyear
+Newtown
+Newill
+Newcom
+Neverson
+Neuhart
+Neuenschwande
+Nestler
+Nenno
+Nejman
+Neiffer
+Neidlinger
+Neglia
+Needs
+Nearing
+Nazarian
+Navor
+Nary
+Narayan
+Nangle
+Nakama
+Naish
+Naik
+Nadolski
+Muscato
+Murphrey
+Murdick
+Murchie
+Muratalla
+Munnis
+Mundwiller
+Muncey
+Munce
+Mullenbach
+Mulhearn
+Mulcahey
+Muhammed
+Muchow
+Mountford
+Moudry
+Mosko
+Morvay
+Morrical
+Morr
+Moros
+Mormann
+Morgen
+Moredock
+Morden
+Mordarski
+Moravek
+Morandi
+Morale
+Mooradian
+Montejo
+Montegut
+Montan
+Monsanto
+Monford
+Moncus
+Molinas
+Molek
+Mohd
+Moehrle
+Moehring
+Modzeleski
+Model
+Modafferi
+Moala
+Moake
+Miyahira
+Mitani
+Mischel
+Minges
+Minella
+Mimes
+Milles
+Milbrett
+Milanes
+Mikolajczyk
+Mikami
+Meucci
+Metler
+Methven
+Metge
+Messmore
+Messerschmidt
+Mesrobian
+Meservey
+Merseal
+Menor
+Menon
+Menear
+Melott
+Melley
+Melfi
+Meinhart
+Megivern
+Megeath
+Meester
+Meeler
+Meegan
+Medoff
+Medler
+Meckley
+Meath
+Mearns
+Mcquigg
+Mcpadden
+Mclure
+Mckellips
+Mckeithen
+Mcglathery
+Mcginnes
+Mcghan
+Mcdonel
+Mccullom
+Mccraken
+Mccrackin
+Mcconathy
+Mccloe
+Mcclaughry
+Mcclaflin
+Mccarren
+Mccaig
+Mcaulay
+Mcaffee
+Mazzuca
+Maytubby
+Mayner
+Maymi
+Mattiello
+Matthis
+Matthees
+Matthai
+Mathiason
+Mastrogiovann
+Masteller
+Mashack
+Marucci
+Martorana
+Martiniz
+Marter
+Martellaro
+Marsteller
+Marris
+Marrara
+Maroni
+Marolda
+Marocco
+Maritn
+Margo
+Maresh
+Maready
+Marchione
+Marbut
+Maranan
+Maragno
+Mapps
+Manrriquez
+Manny
+Mannis
+Manni
+Mangina
+Manganelli
+Mancera
+Mamon
+Maloch
+Mallozzi
+Maller
+Majchrzak
+Majano
+Mainella
+Mahanna
+Maertens
+Madon
+Macumber
+Macioce
+Machuga
+Machlin
+Machida
+Machala
+Mabra
+Lynne
+Lybbert
+Luvert
+Lutts
+Luttrull
+Lupez
+Lukehart
+Ludewig
+Luchsinger
+Loyal
+Lovecchio
+Louissaint
+Loughney
+Lottie
+Lostroh
+Lose
+Lorton
+Lorette
+Lopeman
+Loparo
+Longs
+Loner
+Londo
+Lombera
+Lokietek
+Loiko
+Lohrenz
+Lohan
+Lofties
+Locklar
+Lockaby
+Lobianco
+Loader
+Loa
+Llano
+Livesey
+Litster
+Liter
+Liske
+Linsky
+Linne
+Lindbeck
+Limes
+Licudine
+Leyua
+Levie
+Letterman
+Leonelli
+Lenzo
+Lenze
+Lents
+Leitao
+Leif
+Leidecker
+Leibold
+Lehne
+Legan
+Legacy
+Lefave
+Leehy
+Ledue
+Lecount
+Lecea
+Leadley
+Lazzara
+Lazcano
+Lazalde
+Layer
+Lavi
+Lavancha
+Lavan
+Lav
+Laude
+Latu
+Latty
+Lato
+Larranaga
+Lapidus
+Lapenta
+Langridge
+Langeveld
+Langel
+Lanes
+Landowski
+Landgren
+Landfried
+Lame
+Lamattina
+Lallier
+Lairmore
+Lahaie
+Lagazo
+Lagan
+Lafoe
+Lafluer
+Laflame
+Lafevers
+Lada
+Lacoss
+Lachney
+Labreck
+Labreche
+Labay
+Laa
+Kwasnik
+Kuzyk
+Kutzner
+Kushnir
+Kusek
+Kurtzman
+Kurian
+Kulhanek
+Kuklinski
+Kuh
+Kueny
+Kuczynski
+Kubitz
+Kuang
+Kruschke
+Krous
+Krompel
+Kritz
+Krimple
+Kriese
+Krenzer
+Kreis
+Kratzke
+Krane
+Krage
+Kraebel
+Kozub
+Kozma
+Kouri
+Koudelka
+Kotcher
+Kotas
+Kostic
+Kosh
+Kosar
+Kopko
+Kopka
+Kooy
+Konigsberg
+Konarski
+Kolmer
+Kohlmeyer
+Kobbe
+Knoop
+Knoedler
+Knocke
+Knipple
+Knippenberg
+Knickrehm
+Kneisel
+Kluss
+Klossner
+Klipfel
+Klawiter
+Klasen
+Kittles
+Kissack
+Kirtland
+Kirschenmann
+Kirckof
+Kiphart
+Kinstler
+Kinion
+Kilton
+Killman
+Kiehl
+Kief
+Kett
+Kesling
+Keske
+Kerstein
+Kepple
+Keneipp
+Kempson
+Kempel
+Kelp
+Kehm
+Kehler
+Keh
+Keeran
+Keedy
+Kebert
+Keast
+Kearbey
+Kawaguchi
+Kaupu
+Kauble
+Katzenbach
+Kate
+Katcher
+Kartes
+Karpowicz
+Karpf
+Karen
+Karban
+Kanzler
+Kanarek
+Kamper
+Kaman
+Kalsow
+Kalafut
+Kaeser
+Kaercher
+Kaeo
+Kaeding
+Jurewicz
+Julson
+Jozwick
+Jollie
+Johnigan
+Johll
+Jochum
+Jewkes
+Jestes
+Jeska
+Jersey
+Jereb
+Jayson
+Jaurez
+Jarecki
+Jansma
+Janosik
+Jandris
+Jamin
+Jahr
+Jacot
+Jabs
+Ivens
+Itson
+Isenhower
+Iovino
+Ionescu
+Ingrum
+Ingels
+Inch
+Imrie
+Imlay
+Ihlenfeld
+Ihde
+Igou
+Ibach
+Huyett
+Hurry
+Huppe
+Hultberg
+Hullihen
+Hugi
+Hueso
+Huesman
+Hsiao
+Hronek
+Hovde
+Housewright
+Houlahan
+Hougham
+Houchen
+Hostler
+Hoster
+Hosang
+Hornik
+Hornes
+Horio
+Honyumptewa
+Honeyman
+Honer
+Hommerding
+Holsworth
+Hollobaugh
+Hollinshead
+Hollands
+Hollan
+Holecek
+Holdorf
+Hokes
+Hogston
+Hoesly
+Hodkinson
+Hodgman
+Hodgens
+Hochstedler
+Hochhauser
+Hobbie
+Hoare
+Hnat
+Hiss
+Hiskey
+Hirschy
+Hinostroza
+Hink
+Hing
+Hillmer
+Hillian
+Hillerman
+Hietala
+Hierro
+Hickling
+Hickingbottom
+Heye
+Heubusch
+Hesselschward
+Herriot
+Hernon
+Hermida
+Hermans
+Hentschel
+Henningson
+Henneke
+Henk
+Heninger
+Heltsley
+Helmle
+Helminiak
+Helmes
+Hellner
+Hellmuth
+Helke
+Heitmeyer
+Heird
+Heinle
+Heinicke
+Heinandez
+Heimsoth
+Heimlich
+Heibel
+Hegyi
+Heggan
+Hefel
+Heeralall
+Hedrington
+Heacox
+Hazlegrove
+Hazelett
+Haymore
+Havenhill
+Hautala
+Hascall
+Harvie
+Hartrick
+Hartling
+Harrer
+Harles
+Hargenrader
+Hanshew
+Hanly
+Hankla
+Hanisch
+Hancox
+Hammann
+Hambelton
+Halseth
+Hallisey
+Halleck
+Hallas
+Haisley
+Hairr
+Hainey
+Hainer
+Hailstock
+Haertel
+Guzek
+Guyett
+Guster
+Gussler
+Gurwitz
+Gurka
+Gunsolus
+Guinane
+Guiden
+Gugliotti
+Guevin
+Guevarra
+Guerard
+Gudaitis
+Guadeloupe
+Gschwind
+Grupe
+Grumbach
+Gruenes
+Gruenberg
+Grosser
+Grom
+Grodski
+Groden
+Grizzel
+Gritten
+Griswald
+Grishaber
+Grinage
+Grimwood
+Grims
+Griffon
+Griffies
+Gribben
+Grew
+Gressley
+Gren
+Greenstreet
+Grealish
+Gravett
+Grantz
+Granfield
+Granade
+Gowell
+Gossom
+Gorsky
+Goring
+Goodnow
+Goodfriend
+Goodemote
+Golob
+Gollnick
+Golladay
+Goldwyn
+Goldsboro
+Golds
+Goldrick
+Gohring
+Gohn
+Goettsch
+Goertzen
+Goelz
+Godinho
+Goans
+Glumac
+Gleisner
+Gleen
+Glassner
+Glanzer
+Gladue
+Gjelaj
+Givhan
+Girty
+Girone
+Girgenti
+Giorgianni
+Gilpatric
+Gillihan
+Gillet
+Gilbar
+Gierut
+Gierhart
+Gibert
+Gianotti
+Giannetto
+Gianelli
+Giambanco
+Gharing
+Geurts
+Gettis
+Gettel
+Gest
+Germani
+Gerdis
+Gerbitz
+Geppert
+Gennings
+Gemmer
+Gelvin
+Gellert
+Gehler
+Geddings
+Gearon
+Geach
+Gazaille
+Gayheart
+Gauld
+Gaukel
+Gaudio
+Gato
+Gathing
+Gasque
+Garstka
+Garsee
+Garringer
+Garofano
+Garo
+Garnsey
+Garigen
+Garcias
+Garbe
+Ganoung
+Ganfield
+Ganaway
+Gamero
+Galuska
+Galster
+Gallacher
+Galinski
+Galimi
+Galik
+Galeazzi
+Galdo
+Galdames
+Galas
+Galanis
+Gaglio
+Gaff
+Gaeddert
+Gadapee
+Fussner
+Furukawa
+Fuhs
+Fuerte
+Fuerstenberg
+Fryrear
+Fruits
+Froese
+Fringer
+Frieson
+Friesenhahn
+Frieler
+Friede
+Freymuth
+Freyman
+Freudenberg
+Freman
+Fredricksen
+Frech
+Frasch
+Frantum
+Frankin
+Franca
+Frago
+Fragnoli
+Fouquet
+Fossen
+Foskett
+Forner
+Formosa
+Formisano
+Forget
+Fooks
+Fons
+Folino
+Flott
+Floor
+Flesch
+Flener
+Flemmons
+Flattery
+Flanagin
+Flamino
+Flamand
+Fitzerald
+Findling
+Filsinger
+Fillyaw
+Fillinger
+Fiechter
+Ferre
+Ferdon
+Feldkamp
+Fazzio
+Favia
+Faulconer
+Faughnan
+Faubel
+Fassler
+Faso
+Farrey
+Farrare
+Farnworth
+Farland
+Fairrow
+Faille
+Faherty
+Fagnant
+Fabula
+Fabbri
+Eylicio
+Esteve
+Estala
+Espericueta
+Escajeda
+Erlich
+Equia
+Epson
+Enrriquez
+Enomoto
+Enmon
+Engemann
+Emmerson
+Emmel
+Emler
+Emilio
+Elstad
+Ellwein
+Ellerson
+Eliott
+Eliassen
+Elchert
+Eisenbeis
+Eisel
+Eikenberry
+Eichholz
+Ehmer
+Edris
+Edgerson
+Echenique
+Eberley
+Eans
+Dziuk
+Dykhouse
+Dworak
+Dutt
+Dupas
+Duntz
+Dunshee
+Dunovant
+Dunnaway
+Dummermuth
+Duerson
+Duddy
+Ducotey
+Duchon
+Duchesneau
+Ducci
+Dubord
+Duberry
+Dubach
+Drummonds
+Droege
+Drish
+Drier
+Drexel
+Dresch
+Dresbach
+Drenner
+Drechsler
+Dowen
+Dotter
+Dosreis
+Doser
+Dorward
+Dorin
+Dorf
+Door
+Domeier
+Doler
+Doleman
+Dolbow
+Dolbin
+Dobrunz
+Dobransky
+Dobberstein
+Dlouhy
+Diosdado
+Dingmann
+Dimmer
+Dimarino
+Dimaria
+Dilly
+Dillenburg
+Dilaura
+Dieken
+Dickhaus
+Dibbles
+Dibben
+Diamante
+Dewilde
+Dewaard
+Devich
+Devenney
+Devaux
+Dettinger
+Desroberts
+Dershem
+Dersch
+Derita
+Derickson
+Depina
+Deorio
+Deoliveira
+Denzler
+Dentremont
+Denoble
+Demshar
+Demond
+Demint
+Demichele
+Demel
+Delzer
+Delval
+Delorbe
+Delli
+Delbridge
+Delanoy
+Delancy
+Delahoya
+Dekle
+Deitrick
+Deis
+Dehnert
+Degrate
+Defrance
+Deetz
+Deeg
+Decoster
+Decena
+Dearment
+Daughety
+Datt
+Darrough
+Danzer
+Dante
+Danielovich
+Dandurand
+Dancause
+Dalo
+Dalgleish
+Daisley
+Daft
+Dadlani
+Daddona
+Daddio
+Dacpano
+Cyprian
+Cutillo
+Cush
+Curz
+Curvin
+Cuna
+Cumber
+Cullom
+Cudworth
+Cubas
+Crysler
+Cryderman
+Crummey
+Crumbly
+Crookshanks
+Croes
+Criscione
+Crimes
+Crespi
+Cresci
+Creaser
+Craton
+Cramp
+Cradle
+Cowin
+Cowdrey
+Coutcher
+Cotterman
+Cosselman
+Cosgriff
+Cortner
+Corsini
+Corporan
+Corniel
+Cornick
+Cordts
+Cordial
+Copening
+Coolman
+Connick
+Conlisk
+Conelli
+Common
+Comito
+Colten
+Colling
+Colletta
+Coldivar
+Colclasure
+Colantuono
+Colaizzi
+Coggeshall
+Cockman
+Cockfield
+Cobourn
+Cobo
+Cobarrubias
+Clyatt
+Cloney
+Clonch
+Climes
+Cleckner
+Clearo
+Claybourne
+Clavin
+Claridge
+Claffey
+Ciufo
+Cisnero
+Cipollone
+Cieslik
+Ciejka
+Cichocki
+Cicchetti
+Cianflone
+Chrusciel
+Christesen
+Chmielowiec
+Chirino
+Chillis
+Chihuahua
+Chhoun
+Chevas
+Chehab
+Chaviano
+Chavaria
+Chasten
+Charbonnet
+Chanley
+Champoux
+Champa
+Chalifoux
+Cerio
+Cedotal
+Cech
+Cavett
+Cavendish
+Catoire
+Castronovo
+Castellucci
+Castellow
+Castaner
+Casso
+Cassels
+Cassatt
+Cassar
+Cashon
+Cartright
+Carros
+Carrisalez
+Carrig
+Carrejo
+Carnicelli
+Carnett
+Carlise
+Carline
+Carhart
+Caren
+Cardova
+Cardell
+Carchi
+Caram
+Caquias
+Capper
+Capizzi
+Capano
+Cannedy
+Campese
+Calvello
+Callon
+Callins
+Callies
+Callicutt
+Calix
+Calin
+Califf
+Calderaro
+Caldeira
+Cadriel
+Cadmus
+Cadman
+Caccamise
+Buys
+Buttermore
+Butay
+Bustamente
+Busa
+Burmester
+Burkard
+Burhans
+Burgert
+Bure
+Burdin
+Bullman
+Bulin
+Buelna
+Buehner
+Budin
+Buco
+Buckhanon
+Bryars
+Brutger
+Brus
+Brumitt
+Brum
+Bruer
+Brucato
+Broyhill
+Broy
+Brownrigg
+Brownie
+Brossart
+Brookings
+Broden
+Brocklehurst
+Brockert
+Bristo
+Briskey
+Brisbane
+Bringle
+Bries
+Briar
+Bressman
+Bren
+Branyan
+Brands
+Bramson
+Brammell
+Brallier
+Bozich
+Boysel
+Bowthorpe
+Bowron
+Bowin
+Boutilier
+Boulos
+Boullion
+Boughter
+Bottiglieri
+Borruso
+Borrow
+Borreggine
+Borns
+Borkoski
+Borghese
+Borenstein
+Boran
+Bora
+Booton
+Bonvillain
+Bonini
+Bong
+Bonello
+Bolls
+Boitnott
+Boike
+Bohnet
+Bohnenkamp
+Bohmer
+Boeson
+Boeneke
+Bodey
+Bocchino
+Bobrowski
+Bobic
+Bluestein
+Bloomingdale
+Blogg
+Blewitt
+Blenman
+Bleck
+Blaszak
+Blankenbeckle
+Blando
+Blanchfield
+Blancato
+Blalack
+Blakenship
+Blackett
+Bisping
+Birkner
+Birckhead
+Bingle
+Bineau
+Billiel
+Bigness
+Bies
+Bierer
+Bhalla
+Beyerlein
+Bew
+Betesh
+Besler
+Berzins
+Bertalan
+Berntsen
+Berna
+Bergo
+Berganza
+Bennis
+Benney
+Benkert
+Benjamen
+Benincasa
+Bengochia
+Bendle
+Bendana
+Benchoff
+Benbrook
+Belsito
+Belshaw
+Belinsky
+Belak
+Bela
+Beigert
+Beidleman
+Behen
+Befus
+Beel
+Beebee
+Bedonie
+Beckstrand
+Beckerle
+Beato
+Bears
+Bauguess
+Baughan
+Bauerle
+Battis
+Batis
+Bastone
+Bastille
+Bassetti
+Bashor
+Bary
+Bartunek
+Bartoletti
+Barro
+Barno
+Barnicle
+Barlage
+Barkus
+Barkdull
+Bari
+Barcellos
+Barbarino
+Baranski
+Baranick
+Bankert
+Banchero
+Ban
+Bambrick
+Bamberg
+Bambenek
+Balthrop
+Balmaceda
+Ballman
+Balistrieri
+Balcomb
+Balboni
+Balbi
+Bakshi
+Bagner
+Bagent
+Badasci
+Bacot
+Bache
+Babu
+Babione
+Babic
+Babers
+Babbs
+Awkward
+Avitabile
+Avers
+Avena
+Avance
+Ausley
+Auker
+Audas
+Aud
+Aubut
+Athearn
+Atcheson
+Astorino
+Asplund
+Aslanian
+Askari
+Ashmead
+Asby
+Asai
+Arterbury
+Artalejo
+Arqueta
+Arquero
+Arostegui
+Arnell
+Armeli
+Arista
+Arender
+Arca
+Arballo
+Aprea
+Applen
+Applegarth
+Apfel
+Antonello
+Antolin
+Antkowiak
+Angis
+Angione
+Angerman
+Angelilli
+Andujo
+Andrick
+Anderberg
+Amigon
+Ambers
+Amalfitano
+Alviso
+Alvez
+Altice
+Altes
+Almarez
+Allton
+Allston
+Allgeyer
+Allegretti
+Aliaga
+Algood
+Alberg
+Albarez
+Albaladejo
+Akre
+Aitkin
+Ahles
+Ahlberg
+Agnello
+Adrien
+Adinolfi
+Adamis
+Abramek
+Abolt
+Abitong
+Zurich
+Zurawski
+Zufall
+Zubke
+Zizzo
+Zipperer
+Zinner
+Zinda
+Ziller
+Zill
+Zevallos
+Zesati
+Zenzen
+Zentner
+Zellmann
+Zelinsky
+Zboral
+Zarcone
+Zapalac
+Zaldana
+Zakes
+Zaker
+Zahniser
+Zacherl
+Zabawa
+Zabaneh
+Yum
+Youse
+Youree
+Younis
+Yorty
+Yonce
+Yero
+Yerkey
+Yeck
+Yeargan
+Yauch
+Yashinski
+Yambo
+Xiang
+Wrinn
+Wrightsman
+Worton
+Wortley
+Worland
+Woolworth
+Woolfrey
+Woodhead
+Woltjer
+Wolfenden
+Wolden
+Wolchesky
+Wojick
+Woessner
+Witwer
+Witters
+Witchard
+Wissler
+Wisnieski
+Wisinski
+Winnike
+Winkowski
+Winkels
+Wingenter
+Wineman
+Winegardner
+Wimpy
+Wilridge
+Wilmont
+Willy
+Willians
+Williamsen
+Wilhide
+Wilhelmsen
+Wilhelmi
+Wildrick
+Wilden
+Wiland
+Wiker
+Wigglesworth
+Wiebusch
+Widdowson
+Wiant
+Wiacek
+Whittet
+Whitter
+Whitelock
+Whiteis
+Whiley
+Westrope
+Westpfahl
+Westin
+Wessman
+Wessinger
+Wesemann
+Wesby
+Wertheimer
+Weppler
+Wenke
+Wengler
+Wender
+Welp
+Weitzner
+Weissberg
+Weisenborn
+Weipert
+Weiman
+Weidmann
+Wehrsig
+Wehrenberg
+Weemes
+Weeman
+Wayner
+Waston
+Wasicek
+Wascom
+Wasco
+Warmath
+Warbritton
+Waltner
+Wallenstein
+Waldoch
+Waldal
+Wala
+Waide
+Wadlinger
+Wadhams
+Vullo
+Voorheis
+Vonbargen
+Volner
+Vollstedt
+Vollman
+Vold
+Voge
+Vittorio
+Virtue
+Virginia
+Violett
+Viney
+Vinciguerra
+Vinal
+Villata
+Villarrvel
+Vilanova
+Vigor
+Vigneault
+View
+Vielma
+Veyna
+Vessella
+Versteegh
+Verderber
+Venier
+Venice
+Venditti
+Velotta
+Vejarano
+Veil
+Vecchia
+Vecchi
+Vastine
+Vasguez
+Varella
+Vanry
+Vannah
+Vanhyning
+Vanhuss
+Vanhoff
+Vanhoesen
+Vandivort
+Vandevender
+Vanderlip
+Vanderkooi
+Vandebrink
+Vancott
+Vallien
+Vallas
+Vallandingham
+Valiquette
+Valasek
+Vahey
+Vagott
+Uyematsu
+Urbani
+Uran
+Upp
+Uno
+Union
+Umbach
+Udo
+Tyon
+Tyma
+Twyford
+Twombley
+Twohig
+Tutterrow
+Turnes
+Turkington
+Turchi
+Tunks
+Tumey
+Tumbaga
+Tuinstra
+Tsukamoto
+Tschetter
+Trussel
+Trubey
+Trovillion
+Troth
+Trostel
+Tron
+Trinka
+Trine
+Tribbey
+Triarsi
+Trevor
+Treto
+Trautz
+Tragesser
+Tooman
+Toolson
+Tonozzi
+Tomkiewicz
+Tomb
+Tomasso
+Tolin
+Tolfree
+Toelle
+Tisor
+Tiry
+Tinstman
+Timmermann
+Tillie
+Tickner
+Tiburcio
+Thunberg
+Thronton
+Thompsom
+Theil
+Thayne
+Thaggard
+Teschner
+Tensley
+Tenery
+Tempest
+Tellman
+Tellado
+Telep
+Teigen
+Teator
+Teall
+Tayag
+Tavis
+Tattersall
+Tassoni
+Tarshis
+Tappin
+Tappe
+Tansley
+Talone
+Talford
+Tainter
+Taha
+Taguchi
+Tacheny
+Tabak
+Szymczyk
+Szwaja
+Szopinski
+Sze
+Syvertsen
+Swogger
+Switcher
+Swist
+Swilling
+Swierczek
+Swiech
+Swickard
+Swiatek
+Swezey
+Swepson
+Sweezy
+Swaringen
+Swanagan
+Swailes
+Swade
+Sveum
+Svenningsen
+Svec
+Suttie
+Supry
+Sunga
+Summerhill
+Summars
+Sulit
+Stys
+Stutesman
+Stupak
+Stumpo
+Stuller
+Stuekerjuerge
+Stuckett
+Stuckel
+Stuchlik
+Stuard
+Strutton
+Strop
+Stromski
+Stroebel
+Strehlow
+Strause
+Strano
+Straney
+Stradling
+Stoyle
+Stormo
+Stopyra
+Stoots
+Stoop
+Stonis
+Stoltenburg
+Stoiber
+Stoessel
+Stitzer
+Stien
+Stichter
+Stezzi
+Stewert
+Stepler
+Steinkraus
+Stegemann
+Steeples
+Steenburg
+Steeley
+Staszak
+Stasko
+Starkson
+Stanwick
+Stanke
+Stanifer
+Stangel
+Stain
+Stai
+Squiers
+Sprout
+Springsteen
+Spraglin
+Spragins
+Spraberry
+Spoelstra
+Spisak
+Spirko
+Spille
+Spidel
+Speyer
+Speroni
+Spenst
+Speak
+Spartz
+Sparlin
+Sparacio
+Spaman
+Spainhower
+Sow
+Souers
+Souchet
+Sosbee
+Sorn
+Sorice
+Sorbo
+Soqui
+Somer
+Solon
+Soehl
+Sodergren
+Socorro
+Sobie
+Smucker
+Smsith
+Smoley
+Smolensky
+Smolenski
+Smolder
+Smethers
+Slusar
+Slowey
+Slonski
+Slemmons
+Slatkin
+Slates
+Slappy
+Slaney
+Slagter
+Slacum
+Skutnik
+Skrzypek
+Skibbe
+Sjostrom
+Sjoquist
+Sivret
+Sitko
+Sisca
+Sinnett
+Sineath
+Simoni
+Simar
+Simao
+Silvestro
+Silleman
+Silkwood
+Silha
+Silfies
+Silberhorn
+Silacci
+Sigrist
+Sieczkowski
+Sieczka
+Shure
+Shulz
+Shugrue
+Shrode
+Shown
+Shovlin
+Shortell
+Shonka
+Shiyou
+Shiraishi
+Shiplett
+Sheu
+Shermer
+Sherick
+Sheng
+Sheeks
+Shed
+Sharron
+Shantz
+Shakir
+Shaheed
+Shadoan
+Shadid
+Shackford
+Shabot
+Seung
+Seufert
+Setty
+Setters
+Servis
+Server
+Serres
+Serrell
+Serpico
+Serpas
+Serafine
+Sensenig
+Senft
+Semenec
+Semen
+Semas
+Semaan
+Selvera
+Sellmeyer
+Sek
+Segar
+Seever
+Seeney
+Seeliger
+Seehafer
+Seebach
+Sebben
+Seaward
+Seary
+Searl
+Searby
+Scotland
+Scordino
+Scolieri
+Scolaro
+Schwiebert
+Schwartze
+Schwaner
+Schuur
+Schupbach
+Schumacker
+Schum
+Schudel
+Schubbe
+Schroader
+Schramel
+Schollmeyer
+Schoenherr
+Schoeffler
+Schoeder
+Schnurr
+Schnorr
+Schneeman
+Schnake
+Schnaible
+Schmaus
+Schlotter
+Schinke
+Schimming
+Schimek
+Schikora
+Scheulen
+Scherping
+Schermer
+Scherb
+Schember
+Schellhase
+Schedler
+Schanck
+Schaffhauser
+Schaffert
+Schadler
+Scarola
+Scarfo
+Scarff
+Scantling
+Scaff
+Sayward
+Sayas
+Saxbury
+Savin
+Savel
+Savastano
+Savannah
+Sault
+Satre
+Sarkar
+Santellan
+Sandmeier
+Sampica
+Salvesen
+Saltis
+Salloum
+Salling
+Salce
+Salatino
+Salata
+Salamy
+Safe
+Sadowsky
+Sadlier
+Sabbatini
+Sabatelli
+Sabal
+Sabados
+Rydzewski
+Rybka
+Rybczyk
+Ruz
+Rusconi
+Rupright
+Rufino
+Ruffalo
+Rudiger
+Rudig
+Ruda
+Rubyor
+Royea
+Roxberry
+Rover
+Rouzer
+Roumeliotis
+Roston
+Rossmann
+Rosko
+Rosetta
+Rosene
+Rosenbluth
+Roseland
+Rosasco
+Rosano
+Rosal
+Rorabaugh
+Romie
+Romaro
+Rolstad
+Rollow
+Rohrich
+Roghair
+Rogala
+Roets
+Roen
+Roemmich
+Roelfs
+Roeker
+Roedl
+Roedel
+Rodeheaver
+Roddenberry
+Rockstad
+Rocchi
+Robirds
+Robben
+Robasciotti
+Robaina
+Rizzotto
+Rizzio
+Rittle
+Ritcher
+Rissman
+Riseden
+Ripa
+Rion
+Rintharamy
+Rinehimer
+Rinck
+Riling
+Rike
+Rietschlin
+Riesenberg
+Riemenschneid
+Rieland
+Rickenbaugh
+Rickenbach
+Riches
+Rhody
+Revells
+Reutter
+Respress
+Resnik
+Renton
+Remmel
+Reitmeyer
+Reitan
+Reister
+Reinstein
+Reino
+Reinkemeyer
+Reifschneider
+Reierson
+Reichle
+Rehmeier
+Rehl
+Regine
+Reeds
+Rede
+Records
+Recar
+Rebeiro
+Raybourn
+Rawl
+Rautio
+Raugust
+Raudenbush
+Raudales
+Rattan
+Rashad
+Rapuano
+Rapoport
+Rantanen
+Ransbottom
+Raner
+Ramkissoon
+Rambousek
+Raio
+Rainford
+Radakovich
+Rad
+Rabenhorst
+Quivers
+Quispe
+Quintin
+Quinoes
+Quince
+Quilici
+Quattrone
+Quates
+Quance
+Quale
+Purswell
+Purpora
+Pulera
+Pulcher
+Puckhaber
+Pryer
+Pruyne
+Pruit
+Prudencio
+Prows
+Protzman
+Prothero
+Prospero
+Prosperi
+Prospal
+Privott
+Pritchet
+Priem
+Prest
+Prell
+Preer
+Pree
+Preddy
+Preda
+Pravata
+Pradhan
+Potocki
+Postier
+Postema
+Posse
+Posadas
+Poremba
+Popper
+Popichak
+Ponti
+Pomrenke
+Pomponi
+Pomarico
+Pollok
+Polkinghorn
+Polino
+Pock
+Plough
+Plenty
+Plater
+Plagman
+Pipher
+Pinzone
+Pinkleton
+Pillette
+Pillers
+Pill
+Pilapil
+Pignone
+Pignatelli
+Piersol
+Piepho
+Picton
+Pickrel
+Picket
+Pichard
+Picchi
+Piatek
+Pharo
+Phanthanouvon
+Pettingill
+Pettinato
+Petrovits
+Pethtel
+Petersheim
+Pershing
+Perrez
+Perra
+Pergram
+Peretz
+Perego
+Perches
+Pennello
+Pennella
+Pennant
+Pendry
+Penaz
+Pellish
+Peeks
+Pecanty
+Peare
+Paysour
+Pavlovich
+Pavick
+Pavelko
+Paustian
+Patzer
+Patsy
+Patete
+Patadia
+Paszkiewicz
+Pase
+Pasculli
+Pascascio
+Parrotte
+Parlor
+Parajon
+Paparo
+Papandrea
+Paone
+Pantaleon
+Panning
+Paniccia
+Pancho
+Panarello
+Palmeter
+Pallan
+Palardy
+Pahmeier
+Padget
+Padel
+Oyster
+Oya
+Oxborrow
+Oveson
+Outwater
+Ottaway
+Otake
+Ostermeyer
+Osmer
+Osinski
+Osiecki
+Oroak
+Orndoff
+Orms
+Orkin
+Oregon
+Ordiway
+Opatz
+Onsurez
+Onishi
+Oliger
+Okubo
+Okoye
+Ohlmann
+Offord
+Offner
+Offerdahl
+Oesterle
+Oesch
+Odonnel
+Odeh
+Odebralski
+Obie
+Obermeier
+Oberhausen
+Obenshain
+Obenchain
+Oats
+Nute
+Nulty
+Norrington
+Norlin
+Nore
+Nordling
+Nordhoff
+Norder
+Nordan
+Norals
+Nogales
+Noboa
+Nitsche
+Niermann
+Nienhaus
+Niedringhaus
+Niedbalski
+Nicolella
+Nicolais
+Nickleberry
+Nicewander
+Newfield
+Neurohr
+Neumeier
+Netterville
+Nersesian
+Nern
+Nerio
+Nerby
+Nerbonne
+Neitz
+Neighbours
+Neighbor
+Neidecker
+Neat
+Neason
+Nead
+Navratil
+Naves
+Nastase
+Nasir
+Nasca
+Narine
+Narimatsu
+Nard
+Narayanan
+Nappo
+Namm
+Nalbone
+Nakonechny
+Nabarro
+Myott
+Muthler
+Muscatello
+Murriel
+Murin
+Murders
+Muoio
+Mundel
+Munafo
+Mulch
+Mukherjee
+Muffoletto
+Muessig
+Muckey
+Mucher
+Mruk
+Moyd
+Mowell
+Mowatt
+Moutray
+Mourning
+Mou
+Motzer
+Moster
+Mortis
+Morgenroth
+Morga
+Morataya
+Montross
+Montezuma
+Monterroza
+Montemarano
+Montello
+Montbriand
+Montavon
+Montaque
+Monigold
+Monforte
+Molgard
+Moleski
+Mohsin
+Mohead
+Mofield
+Moerbe
+Moeder
+Mochizuki
+Miyazaki
+Miyasaki
+Mital
+Miskin
+Mischler
+Minus
+Minniear
+Minero
+Milosevic
+Mildenhall
+Mila
+Mikhail
+Mielsch
+Midden
+Michonski
+Michniak
+Michitsch
+Michelotti
+Micheli
+Michelfelder
+Michand
+Miao
+Metelus
+Merkt
+Merando
+Meranda
+Mentz
+Meneley
+Menaker
+Memory
+Melino
+Meir
+Mehaffy
+Meehl
+Meech
+Meczywor
+Mcweeney
+Mcumber
+Mcredmond
+Mcneer
+Mcnay
+Mcmikle
+Mcmaken
+Mclaurine
+Mclauglin
+Mclaney
+Mckune
+Mckinnies
+Mckague
+Mchattie
+Mcgrapth
+Mcglothen
+Mcgath
+Mcfolley
+Mcdannell
+Mccurty
+Mccort
+Mcclymonds
+Mcclimon
+Mcclamy
+Mccaughan
+Mccartan
+Mccan
+Mccadden
+Mcburnie
+Mcburnett
+Mcbryar
+Mcannally
+Mcalevy
+Mcaleese
+Maytorena
+Mayrant
+Mayol
+Mayland
+Mayeaux
+Mauter
+Matthewson
+Mathiew
+Matern
+Matera
+Maslow
+Mashore
+Masaki
+Maruco
+Martorell
+Martenez
+Marry
+Marrujo
+Marrison
+Maroun
+Markway
+Markos
+Markoff
+Markman
+Marian
+Marello
+Marbry
+Marban
+Maranda
+Maphis
+Manuele
+Mansel
+Manganello
+Mandrell
+Mandoza
+Manard
+Manago
+Maltba
+Mallick
+Mallak
+Maline
+Malikowski
+Majure
+Majcher
+Maise
+Mahl
+Maffit
+Maffeo
+Madueno
+Madlem
+Madariaga
+Macvane
+Mackler
+Macconnell
+Macchi
+Maccarone
+Lyng
+Lynchard
+Lura
+Lunning
+Luneau
+Lunden
+Lumbra
+Lumbert
+Lueth
+Ludington
+Luckado
+Lucchini
+Lucatero
+Luallen
+Lozeau
+Lowen
+Lovera
+Lovelock
+Louck
+Lothian
+Lorio
+Lorimer
+Lorge
+Loretto
+Longhenry
+Lonas
+Loiseau
+Lohrman
+Logel
+Loft
+Locks
+Lockie
+Llerena
+Livington
+Liuzzi
+Liscomb
+Lippeatt
+Liou
+Linhardt
+Lindelof
+Lindbo
+Limehouse
+Limage
+Lillo
+Lillian
+Lilburn
+Liggons
+Lidster
+Liddy
+Liddick
+Lich
+Liberato
+Lian
+Lia
+Leysath
+Lewelling
+Lesney
+Leser
+Lescano
+Leonette
+Lentsch
+Lenius
+Lemmo
+Lemming
+Lemcke
+Lein
+Leggette
+Legerski
+Legard
+Leever
+Leete
+Ledin
+Lecomte
+Lecocq
+Leakes
+Leab
+Lazarz
+Layous
+Lawrey
+Lawery
+Lauze
+Lautz
+Laughinghouse
+Latulippe
+Lattus
+Lattanzio
+Later
+Lascano
+Larmer
+Laris
+Larcher
+Laprise
+Lapin
+Lapage
+Lano
+Langseth
+Langman
+Langland
+Landstrom
+Landsberg
+Landsaw
+Landram
+Lamphier
+Lamendola
+Lamberty
+Lakhani
+Laker
+Lajara
+Lagrow
+Lagman
+Ladewig
+Laderman
+Ladden
+Lacrue
+Laclaire
+Lachut
+Lachner
+Kwit
+Kvamme
+Kvam
+Kutscher
+Kushi
+Kurgan
+Kunsch
+Kundert
+Kun
+Kulju
+Kukene
+Kudo
+Kubin
+Kubes
+Kuberski
+Krystofiak
+Kruppa
+Krul
+Krukowski
+Kruegel
+Kronemeyer
+Krock
+Kriston
+Kretzer
+Krenn
+Kralik
+Krafft
+Krabill
+Kozisek
+Kovich
+Koverman
+Kovatch
+Kovarik
+Kotlowski
+Kosmala
+Kosky
+Kosir
+Kosa
+Korpi
+Kornbluth
+Koppen
+Kooistra
+Kohlhepp
+Kofahl
+Koeneman
+Koebel
+Koczur
+Kobrin
+Kobashigawa
+Koba
+Knuteson
+Knoff
+Knoble
+Knipper
+Knierim
+Kneisley
+Klusman
+Kloc
+Klitzing
+Klinko
+Klinefelter
+Klemetson
+Kleinpeter
+Klauser
+Klatte
+Klaren
+Klare
+Kissam
+Kirkhart
+Kirchmeier
+Kinzinger
+Kindt
+Kincy
+Kincey
+Kimoto
+Killingworth
+Kilcullen
+Kilbury
+Kietzman
+Kienle
+Kiedrowski
+Kidane
+Khamo
+Khalili
+Ketterling
+Ketchem
+Kessenich
+Kessell
+Kepp
+Kenon
+Kenning
+Kennady
+Kendzior
+Kemppainen
+Kellermann
+Keirns
+Keilen
+Keiffer
+Kehew
+Keelan
+Keawe
+Keator
+Kealy
+Keady
+Kathman
+Kastler
+Kastanes
+Kassab
+Karren
+Karpin
+Karau
+Karathanasis
+Kara
+Kaps
+Kaplun
+Kapaun
+Kannenberg
+Kanipe
+Kander
+Kandel
+Kanas
+Kanan
+Kamke
+Kaltenbach
+Kallenberger
+Kallam
+Kali
+Kaley
+Kafton
+Kafer
+Kabler
+Kaaihue
+Jupiter
+Jundt
+Jubilee
+Jovanovich
+Jojola
+Johnstad
+Jodon
+Joachin
+Jinright
+Jew
+Jessick
+Jeronimo
+Jerald
+Jenne
+Jelsma
+Jeannotte
+Jeangilles
+Jaworsky
+Jaubert
+Jarry
+Jarrette
+Jarreau
+Jarett
+Janos
+Janecka
+Janczak
+Jalomo
+Jagoda
+Jagla
+Jacquier
+Jaber
+Iwata
+Ivanoff
+Isola
+Iserman
+Isais
+Isaacks
+Iron
+Inverso
+Infinger
+Ibsen
+Hyser
+Hylan
+Hybarger
+Hwee
+Hutchenson
+Hutchcroft
+Husar
+Hurlebaus
+Hunsley
+Hunker
+Hummingbird
+Humberson
+Hulst
+Hulon
+Huhtala
+Hugill
+Hugghins
+Huffmaster
+Huckeba
+Hrabovsky
+Howden
+Hoverson
+Houts
+Houskeeper
+Housh
+Hosten
+Horras
+Horchler
+Hor
+Hopke
+Hooke
+Honie
+Holtsoi
+Holsomback
+Holoway
+Holmstead
+Hoistion
+Hohnstein
+Hoheisel
+Hoguet
+Hoggle
+Hogenson
+Hoffstetter
+Hoffler
+Hoffa
+Hofe
+Hoefling
+Hoague
+Hizer
+Hirschfield
+Hironaka
+Hiraldo
+Hinote
+Hingston
+Hind
+Hinaman
+Hillie
+Hillesheim
+Hilderman
+Hiestand
+Heyser
+Heys
+Hews
+Hew
+Hertler
+Herrero
+Herrandez
+Heppe
+Henle
+Henkensiefken
+Henigan
+Henandez
+Henagan
+Hemberger
+Heman
+Helser
+Helmich
+Hellinger
+Helfrick
+Heldenbrand
+Heinonen
+Heineck
+Heikes
+Heidkamp
+Heglar
+Heffren
+Heelan
+Hedgebeth
+Heckmann
+Heckaman
+Hechmer
+Hazelhurst
+Hawken
+Haverkamp
+Havatone
+Hausauer
+Hasch
+Harwick
+Hartse
+Harts
+Harrower
+Harle
+Hargroder
+Hardway
+Hardinger
+Hardemon
+Harbeck
+Hant
+Hamre
+Hamberg
+Hallback
+Haisten
+Hailstone
+Hahl
+Hagner
+Hagman
+Hagemeyer
+Haeussler
+Hackwell
+Haby
+Haataja
+Gverrero
+Gustovich
+Gustave
+Guske
+Gushee
+Gurski
+Gurnett
+Gura
+Gunto
+Gunselman
+Gugler
+Gudmundson
+Gudinas
+Guarneri
+Grumbine
+Gruis
+Grotz
+Grosskopf
+Grosman
+Grosbier
+Grinter
+Grilley
+Grieger
+Grewal
+Gressler
+Greaser
+Graus
+Grasman
+Graser
+Grannan
+Granath
+Gramer
+Graboski
+Goyne
+Gowler
+Gottwald
+Gottesman
+Goshay
+Gorr
+Gorovitz
+Gores
+Goossens
+Goodier
+Goodhue
+Gonzeles
+Gonzalos
+Gonnella
+Golomb
+Golick
+Golembiewski
+Goeke
+Godzik
+Goar
+Glosser
+Glendenning
+Glendening
+Glatter
+Glas
+Gittings
+Gitter
+Gisin
+Giscombe
+Gimlin
+Gillitzer
+Gillick
+Gilliand
+Gilb
+Gigler
+Gidden
+Gibeau
+Gibble
+Gianunzio
+Giannattasio
+Gertelman
+Gerosa
+Gerold
+Gerland
+Gerig
+Gerecke
+Gerbino
+Genz
+Genovesi
+Genet
+Gelrud
+Geitgey
+Geiszler
+Gehrlein
+Gazzo
+Gawrys
+Gavilanes
+Gaulden
+Gate
+Garthwaite
+Garmoe
+Gargis
+Gara
+Gannett
+Galligher
+Galler
+Galleher
+Gallahan
+Galford
+Gal
+Gahn
+Gacek
+Gabert
+Fuster
+Furuya
+Furse
+Fujihara
+Fuhriman
+Fruit
+Frueh
+Fromme
+From
+Froemming
+Friskney
+Frietas
+Freiler
+Freelove
+Freber
+Frear
+Frankl
+Frankenfield
+Franey
+Francke
+Foxworthy
+Formella
+Foringer
+Forgue
+Forge
+Fonnesbeck
+Fonceca
+Folland
+Fodera
+Fode
+Floresca
+Fleurent
+Fleshner
+Flentge
+Fleischhacker
+Fleeger
+Flecher
+Flam
+Flair
+Flaim
+Fivecoat
+Firebaugh
+Fioretti
+Finucane
+Filley
+Figuroa
+Figuerda
+Fiddelke
+Feurtado
+Fetterly
+Fessel
+Femia
+Feild
+Fehling
+Fegett
+Fedde
+Fechter
+Fawver
+Faustino
+Faulhaber
+Fatchett
+Fassnacht
+Fashaw
+Fasel
+Farrugia
+Farran
+Farness
+Farhart
+Farbman
+Fama
+Falwell
+Falvo
+Falling
+Falkenstein
+Falin
+Failor
+Faigin
+Fagundo
+Fague
+Fagnan
+Fagerstrom
+Faden
+Eytchison
+Eyles
+Ewy
+Evon
+Everage
+Evangelist
+Estrin
+Estorga
+Esponda
+Espindola
+Escher
+Esche
+Escarsega
+Escandon
+Erven
+Erding
+Eplin
+Enix
+Englade
+Engdahl
+Enck
+Emmette
+Embery
+Emberson
+Eltzroth
+Else
+Elsayed
+Ellerby
+Ellens
+Elhard
+Elfers
+Elazegui
+Eisermann
+Eilertson
+Eiben
+Ehrhard
+Ehresman
+Egolf
+Egnew
+Eggins
+Efron
+Effland
+Eduardo
+Edminster
+Edgeston
+Ede
+Eckstrom
+Eckhard
+Eckford
+Echoles
+Ebsen
+Eatherly
+Eastlick
+Earnheart
+Ear
+Dykhuizen
+Dyas
+Duttweiler
+Dutka
+Dutch
+Dusenbury
+Dusenbery
+Durre
+Durnil
+Durnell
+Durie
+Durhan
+Durando
+Dupriest
+Dunsmoor
+Dunseith
+Dunnum
+Dunman
+Dunlevy
+Duma
+Dulude
+Dulong
+Duignan
+Dugar
+Dufek
+Ducos
+Duchaine
+Duch
+Dubow
+Drowne
+Dross
+Drollinger
+Droke
+Driggars
+Dredge
+Drawhorn
+Drach
+Drabek
+Doyne
+Doukas
+Dorvil
+Dorow
+Doroski
+Dornak
+Dormer
+Dorian
+Donnelson
+Donna
+Donn
+Donivan
+Dondero
+Dompe
+Dolle
+Doakes
+Diza
+Dixie
+Divirgilio
+Ditore
+Distel
+Disimone
+Disbro
+Dipiero
+Dingson
+Diluzio
+Dillehay
+Dilbert
+Digiorgio
+Diflorio
+Dietzler
+Dietsch
+Dieterle
+Dierolf
+Dierker
+Dicostanzo
+Dicesare
+Dexheimer
+Dewitte
+Dewing
+Devoti
+Devincentis
+Devary
+Deutschman
+Dettloff
+Detienne
+Destasio
+Dest
+Despard
+Desmet
+Deslatte
+Desfosses
+Derise
+Derenzo
+Deppner
+Depolo
+Denoyer
+Denoon
+Denno
+Denne
+Deniston
+Denike
+Denes
+Demoya
+Demick
+Demicco
+Demetriou
+Demange
+Delva
+Delorge
+Delley
+Delisio
+Delhoyo
+Delgrande
+Delgatto
+Delcour
+Delair
+Deinert
+Degruy
+Degrave
+Degeyter
+Defino
+Deffenbaugh
+Deener
+Decook
+Decant
+Deboe
+Deblanc
+Deatley
+Dearmitt
+Deale
+Deaguiar
+Dayan
+Daus
+Dauberman
+Datz
+Dase
+Dary
+Dartt
+Darocha
+Dario
+Dari
+Dardis
+Dapper
+Danowski
+Dancel
+Dami
+Dallmann
+Dalere
+Dalba
+Dakan
+Daise
+Dailing
+Dahan
+Dagnan
+Daggs
+Dagan
+Czarkowski
+Czaplinski
+Cutten
+Curtice
+Curenton
+Cure
+Curboy
+Cura
+Culliton
+Culberth
+Cucchiara
+Cubbison
+Csaszar
+Crytser
+Crotzer
+Crossgrove
+Crosser
+Croshaw
+Croissant
+Crocco
+Critzer
+Creveling
+Cressy
+Creps
+Creese
+Cratic
+Crate
+Craigo
+Craigen
+Craib
+Cracchiolo
+Crable
+Coykendall
+Cowick
+Coville
+Couzens
+Coutch
+Cousens
+Cousain
+Counselman
+Coult
+Cotterell
+Cott
+Cotham
+Corsaut
+Corriere
+Corredor
+Cornet
+Cornelia
+Corkum
+Coreas
+Cordoza
+Corbet
+Corathers
+Conwill
+Contreas
+Consuegra
+Constanza
+Conolly
+Conedy
+Companion
+Comins
+Combee
+Colosi
+Colom
+Colmenares
+Collymore
+Colleran
+Colina
+Colaw
+Colatruglio
+Colantro
+Colantonio
+Cohea
+Cogill
+Codner
+Code
+Codding
+Cockram
+Cocanougher
+Cobine
+Cluckey
+Clucas
+Cloward
+Cloke
+Clisham
+Clipper
+Clinebell
+Cliffe
+Clendenen
+Cisowski
+Cirelli
+Ciraolo
+Ciocca
+Cintora
+Ciesco
+Cibrian
+Chupka
+Chugg
+Christmann
+Choma
+Chiverton
+Chirinos
+Chinen
+Chimenti
+Chima
+Cheuvront
+Chesla
+Chesher
+Chesebro
+Chern
+Chehebar
+Cheatum
+Chastine
+Chapnick
+Chapelle
+Chambley
+Cercy
+Celius
+Celano
+Cayea
+Cavicchi
+Cattell
+Catanach
+Catacutan
+Castelluccio
+Castellani
+Cassmeyer
+Cassetta
+Cassada
+Caspi
+Cashmore
+Casebier
+Casanas
+Carrothers
+Carrizal
+Carriveau
+Carretero
+Carradine
+Carosella
+Carnine
+Carmel
+Carloni
+Carkhuff
+Cardosi
+Cardo
+Carchidi
+Caravello
+Caranza
+Carandang
+Capes
+Cantrall
+Canpos
+Canoy
+Cannizzaro
+Canion
+Canida
+Canham
+Cangemi
+Cange
+Candle
+Cancelliere
+Canard
+Camarda
+Calverley
+Calogero
+Callendar
+Calame
+Cadrette
+Cachero
+Caccavale
+Cabreros
+Cabrero
+Cabrara
+Cabler
+Butzer
+Butte
+Butrick
+Butala
+Bustios
+Busser
+Busic
+Bushorn
+Busher
+Burmaster
+Burl
+Burkland
+Burkins
+Burkert
+Burgueno
+Burgraff
+Buren
+Burel
+Burdon
+Burck
+Burby
+Buoy
+Bunk
+Bumford
+Bulock
+Bujnowski
+Buggie
+Buffy
+Budine
+Bucciero
+Bubier
+Brzoska
+Brydges
+Brumlow
+Brosseau
+Brooksher
+Brokke
+Broeker
+Brittin
+Bristle
+Briano
+Briand
+Brettschneide
+Bresnan
+Brentson
+Brenneis
+Brender
+Brazle
+Brassil
+Brasington
+Branstrom
+Branon
+Branker
+Brandwein
+Brandau
+Brana
+Bralley
+Brailey
+Brague
+Brade
+Bozzi
+Bownds
+Bowmer
+Bournes
+Bour
+Bouchey
+Botto
+Boteler
+Borroel
+Borra
+Boroski
+Boothroyd
+Boord
+Bonny
+Bonga
+Bonato
+Bonadonna
+Bolejack
+Boldman
+Boiser
+Boggio
+Bogacki
+Boerboom
+Boehnlein
+Boehle
+Bodah
+Bobst
+Boak
+Bluemel
+Blockmon
+Blitch
+Blincoe
+Bleier
+Blaydes
+Blasius
+Bittel
+Bir
+Binsfeld
+Bindel
+Bilotti
+Billiott
+Bilbrew
+Bihm
+Biersner
+Bielat
+Bidrowski
+Bickler
+Biasi
+Bianca
+Bhola
+Bhat
+Bewick
+Betzen
+Bettridge
+Betti
+Betsch
+Besley
+Beshero
+Besa
+Bertoli
+Berstein
+Berrien
+Berrie
+Berrell
+Bermel
+Berenguer
+Benzer
+Bensing
+Bennie
+Benedix
+Bemo
+Belile
+Beilman
+Behunin
+Behrmann
+Bedient
+Becht
+Beaule
+Beaudreault
+Bealle
+Beagley
+Bayuk
+Bayot
+Bayliff
+Baugess
+Battistoni
+Batrum
+Basinski
+Basgall
+Bartolomei
+Bartnik
+Bartl
+Bartko
+Bartholomay
+Barthlow
+Bartgis
+Barsness
+Barski
+Barlette
+Barickman
+Bargen
+Bardon
+Barcliff
+Barbu
+Barbar
+Barakat
+Baracani
+Baraban
+Banos
+Banko
+Bania
+Bambach
+Balok
+Balogun
+Bally
+Baldini
+Balck
+Balcer
+Balash
+Baim
+Bailor
+Bahm
+Bahar
+Bagshaw
+Baggerly
+Badie
+Badal
+Backues
+Babino
+Ba
+Aydelott
+Awbrey
+Aversano
+Avansino
+Auyon
+Aukamp
+Aujla
+Augenstein
+Astacio
+Ast
+Asplin
+Asato
+Asano
+Aruizu
+Artale
+Arrick
+Arneecher
+Armelin
+Armbrester
+Armacost
+Arkell
+Argue
+Argrave
+Areizaga
+Areas
+Apolo
+Anzures
+Anzualda
+Antwi
+Antillon
+Antenor
+Annand
+Anhalt
+Angove
+Anglemyer
+Anglada
+Angiano
+Angeloni
+Andaya
+Ancrum
+Anagnos
+Ammirati
+Amescua
+America
+Ambrosius
+Amacker
+Amacher
+Amabile
+Alvizo
+Alvernaz
+Alvara
+Altobelli
+Altobell
+Althauser
+Alterman
+Altavilla
+Alsip
+Alphonso
+Almeyda
+Almeter
+Alman
+Allscheid
+Allaman
+Aliotta
+Alicia
+Aliberti
+Alghamdi
+Alfonzo
+Albiston
+Alberta
+Alberding
+Alarie
+Alano
+Aja
+Ailes
+Ahsan
+Ahrenstorff
+Ahler
+Aerni
+Ackland
+Achor
+Acero
+Acebo
+Ace
+Abshier
+Abruzzo
+Abrom
+Abood
+Abnet
+Abend
+Abegg
+Abbruzzese
+Aaberg
+Zysk
+Zutell
+Zumstein
+Zummo
+Zuhlke
+Zuehlsdorff
+Zuch
+Zucconi
+Zortman
+Zohn
+Ziv
+Zingone
+Zingg
+Zingale
+Zima
+Zientek
+Zieg
+Zervas
+Zerger
+Zenk
+Zeldin
+Zeiss
+Zeiders
+Zediker
+Zea
+Zavodny
+Zarazua
+Zappone
+Zappala
+Zapanta
+Zaniboni
+Zanchi
+Zampedri
+Zaller
+Zakrajsek
+Zagar
+Zadrozny
+Zablocki
+Zable
+Yust
+Yunk
+Youngkin
+Yosten
+Yockers
+Yochim
+Yerke
+Yerena
+Yeast
+Yanos
+Yam
+Wysinger
+Wyner
+Wrisley
+Woznicki
+Wortz
+Worsell
+Wooters
+Woon
+Woolcock
+Woodke
+Wonnacott
+Wolnik
+Wittstock
+Witting
+Witry
+Witfield
+Witcraft
+Wissmann
+Wissink
+Wisehart
+Wiscount
+Wironen
+Wipf
+Winterrowd
+Wingett
+Windon
+Windish
+Windisch
+Windes
+Wiltbank
+Willmarth
+Willick
+Wiler
+Wieseler
+Wiedmaier
+Wiederstein
+Wiedenheft
+Wieberg
+Wickware
+Wickkiser
+Wickell
+Whittmore
+Whitker
+Whitegoat
+Whitcraft
+Whisonant
+Whisby
+Whetsell
+Whedon
+Westry
+Westcoat
+Wernimont
+Wentling
+Wendlandt
+Wencl
+Weisgarber
+Weininger
+Weikle
+Weigold
+Weigl
+Weichbrodt
+Wehrli
+Wehe
+Weege
+Weare
+Watland
+Wassmann
+Warzecha
+Warrix
+Warrell
+Warnack
+Waples
+Wantland
+Wanger
+Wandrei
+Wander
+Wanat
+Wampole
+Waltjen
+Walterscheid
+Waligora
+Walding
+Waldie
+Walczyk
+Wakins
+Waitman
+Wair
+Wainio
+Wahpekeche
+Wahlman
+Wagley
+Wagenknecht
+Wadle
+Waddoups
+Wadding
+Wack
+Vuono
+Vuillemot
+Vugteveen
+Vosmus
+Vorkink
+Vories
+Vondra
+Voelz
+Vlashi
+Vivo
+Vitelli
+Vitali
+Viscarra
+Virgo
+Vinet
+Vimont
+Villega
+Villard
+Vignola
+Viereck
+Videtto
+Vicoy
+Vessell
+Vescovi
+Verros
+Vernier
+Vernaglia
+Vergin
+Verdone
+Verdier
+Verastequi
+Vejar
+Vasile
+Vasi
+Varnadore
+Vardaro
+Vanzanten
+Vansumeren
+Vanschuyver
+Vanleeuwen
+Vanhowe
+Vanhoozer
+Vaness
+Vandewalker
+Vandevoorde
+Vandeveer
+Vanderzwaag
+Vanderweide
+Vanderhyde
+Vandellen
+Vanamburg
+Vanalst
+Vallin
+Valk
+Valerie
+Valentini
+Valcarcel
+Valasco
+Valadao
+Vacher
+Urquijo
+Unterreiner
+Unsicker
+Unser
+Unrau
+Undercoffler
+Uhm
+Uffelman
+Uemura
+Ueda
+Tyszko
+Tyska
+Tymon
+Tyce
+Tyacke
+Twinam
+Tutas
+Tussing
+Turmel
+Turkowski
+Turkel
+Turchetta
+Tupick
+Tumblin
+Tukes
+Tufte
+Tufo
+Tuey
+Tuell
+Tuckerman
+Tsutsumi
+Tsuchiya
+Try
+Trossbach
+Trivitt
+Trippi
+Trippensee
+Trimbach
+Trillo
+Triller
+Trible
+Tribe
+Tribby
+Trevisan
+Tresch
+Tramonte
+Traff
+Trad
+Tousey
+Totaro
+Torregrosa
+Torralba
+Torn
+Tolly
+Tofil
+Tofani
+Tobiassen
+Tippy
+Tiogangco
+Tino
+Tinnes
+Tingstrom
+Tingen
+Tine
+Tindol
+Tifft
+Tiffee
+Tiet
+Thuesen
+Thruston
+Throndson
+Thornsbury
+Thornes
+Thiery
+Thielman
+Thie
+Theilen
+Thede
+Thate
+Thane
+Thalacker
+Thaden
+Teuscher
+Terracina
+Terell
+Terada
+Tepfer
+Tennessee
+Tenneson
+Tenant
+Temores
+Temkin
+Tellers
+Telleria
+Teaque
+Tealer
+Teachey
+Tavakoli
+Tauras
+Taucher
+Tator
+Tartaglino
+Tarpy
+Tape
+Tannery
+Tani
+Tams
+Tamlin
+Tambe
+Tallis
+Talamante
+Takayama
+Takaki
+Takagi
+Taibl
+Taffe
+Tadesse
+Tade
+Tabeling
+Tabag
+Szoke
+Szoc
+Szala
+Szady
+Sysak
+Sylver
+Syler
+Swonger
+Swiggett
+Swensson
+Sweis
+Sweers
+Sweene
+Sweany
+Sweaney
+Swartwout
+Swamy
+Swales
+Swab
+Susman
+Surman
+Surgeon
+Sundblad
+Summerset
+Summerhays
+Sumerall
+Sule
+Sugimoto
+Subramanian
+Sturch
+Stupp
+Stunkard
+Stumpp
+Struiksma
+Stropes
+Stromyer
+Stromquist
+Strede
+Strazza
+Strauf
+Storniolo
+Storjohann
+Stonum
+Stonier
+Stonecypher
+Stoneberger
+Stollar
+Stokke
+Stokan
+Stoetzel
+Stoeckel
+Stockner
+Stockinger
+Stockholm
+Stockert
+Stockdill
+Stobbe
+Stitzel
+Stitely
+Stirgus
+Stigers
+Stettner
+Stettler
+Sterlin
+Sterbenz
+Stemp
+Stelluti
+Steinmeyer
+Steininger
+Steinauer
+Steigerwalt
+Steider
+Steady
+Stavrou
+Staufenberger
+Stassi
+Starin
+Stankus
+Stanaway
+Stammer
+Stakem
+Staino
+Stahlnecker
+Stagnitta
+Staelens
+Staal
+Srsen
+Sprott
+Sprigg
+Sprenkle
+Sprenkel
+Spreitzer
+Spraque
+Sprandel
+Spotted
+Sporn
+Spivak
+Spira
+Spiewak
+Spieth
+Spiering
+Sperow
+Speh
+Specking
+Spease
+Spead
+Sparger
+Spanier
+Spall
+Sower
+Southcott
+Sosna
+Soran
+Sookram
+Sonders
+Solak
+Sohr
+Sohl
+Sofranko
+Soderling
+Sochor
+Sobon
+Smutz
+Smudrick
+Smithj
+Smid
+Slosser
+Sliker
+Slenker
+Sleight
+Sleger
+Sleet
+Slaby
+Skousen
+Skilling
+Skibinski
+Skeeters
+Skeet
+Skees
+Skane
+Skafidas
+Sivic
+Sivertsen
+Sivers
+Sitra
+Sito
+Siracusa
+Sinicki
+Simpers
+Simley
+Simbeck
+Silberberg
+Siever
+Siegwarth
+Sidman
+Siddons
+Siddle
+Sibbett
+Si
+Shumard
+Shubrooks
+Shough
+Shorb
+Shoptaw
+Sholty
+Shoffstall
+Shiverdecker
+Shininger
+Shimasaki
+Shifrin
+Shiffler
+Sheston
+Sherr
+Sherill
+Shere
+Shepeard
+Shelquist
+Shells
+Sheler
+Shave
+Shauf
+Sharrar
+Sharpnack
+Shanon
+Shamsiddeen
+Shambley
+Shallenberger
+Shadler
+Shaban
+Sha
+Sferra
+Seys
+Sexauer
+Sevey
+Severo
+Setlak
+Seta
+Sesko
+Sersen
+Serratore
+Serdula
+Senechal
+Seldomridge
+Seilhamer
+Seifer
+Seidlitz
+Sehnert
+Sedam
+Sebron
+Seber
+Sebek
+Seavers
+Sear
+Scullark
+Scroger
+Scovill
+Sciascia
+Sciarra
+Schweers
+Schwarze
+Schummer
+Schultes
+Schuchardt
+Schuchard
+Schrieber
+Schrenk
+Schreifels
+Schowalter
+Schoultz
+Scholer
+Schofill
+Schoff
+Schnuerer
+Schnettler
+Schmitke
+Schmiege
+Schloop
+Schlinger
+Schlessman
+Schlesser
+Schlageter
+Schiess
+Schiefer
+Schiavoni
+Scherzer
+Scherich
+Schechtman
+Schebel
+Scharpman
+Schaich
+Schaap
+Scappaticci
+Scadlock
+Savocchia
+Savini
+Savers
+Save
+Savageau
+Sauvage
+Sause
+Sauerwein
+Sary
+Sarwary
+Sarnicola
+Santone
+Santoli
+Santalucia
+Santacruce
+Sansoucie
+Sankoff
+Sanes
+Sandri
+Sanderman
+Sammartano
+Salmonson
+Salmela
+Salmans
+Sallaz
+Salis
+Sakuma
+Sakowski
+Sajdak
+Sahm
+Sagredo
+Safrit
+Sade
+Sackey
+Sabio
+Sabino
+Sabina
+Rybolt
+Ruzzo
+Ruthstrom
+Ruta
+Russin
+Russian
+Russak
+Rusko
+Ruskin
+Rusiecki
+Ruscher
+Rupar
+Rumberger
+Rullan
+Ruliffson
+Ruhlman
+Ruger
+Rufenacht
+Ruelle
+Rudisell
+Rudi
+Rucci
+Rublee
+Ruberto
+Rubeck
+Rowett
+Rouge
+Rottinghaus
+Roton
+Rothgeb
+Rothgaber
+Rothermich
+Rostek
+Rossini
+Roskelley
+Rosing
+Rosi
+Rosewell
+Rosebush
+Rosberg
+Roon
+Ronin
+Romesburg
+Romelus
+Rolley
+Rollerson
+Rollefson
+Rolins
+Rolens
+Rois
+Rohrig
+Rohrbacher
+Rohland
+Rohen
+Roh
+Rogness
+Roes
+Roering
+Roehrick
+Roebke
+Rodregez
+Rodabaugh
+Rocks
+Rockingham
+Roblee
+Robel
+Roadcap
+Rizzolo
+Riviezzo
+Rivest
+Riveron
+Risto
+Rissler
+Risen
+Rippentrop
+Ripka
+Rinn
+Ringuette
+Ringering
+Rindone
+Rindels
+Rim
+Rieffer
+Riedman
+Riede
+Riecke
+Riebow
+Riddlebarger
+Rhome
+Rhodd
+Rhatigan
+Rhame
+Reyers
+Rewitzer
+Revalee
+Retzer
+Rettinger
+Reschke
+Requa
+Reper
+Reopell
+Renzelman
+Renne
+Renker
+Renk
+Renicker
+Rendina
+Rendel
+Remund
+Remmele
+Remiasz
+Remaklus
+Remak
+Reitsma
+Reitmeier
+Reiswig
+Reishus
+Reining
+Reim
+Reidinger
+Reick
+Reiche
+Regans
+Reffett
+Reesor
+Reekie
+Redpath
+Redditt
+Rechtzigel
+Recht
+Rebel
+Rearden
+Raynoso
+Raxter
+Ratkowski
+Rasulo
+Rassmussen
+Rassel
+Raspberry
+Raser
+Rappleye
+Rappe
+Randy
+Randrup
+Randleman
+Ramson
+Rampey
+Ramming
+Rama
+Rainier
+Raider
+Radziewicz
+Quirarte
+Quintyne
+Quickel
+Query
+Quattrini
+Quarry
+Quakenbush
+Quaile
+Pytel
+Putty
+Pushaw
+Pusch
+Purslow
+Punzo
+Pullam
+Pugmire
+Puello
+Pu
+Przekop
+Pruss
+Pruiett
+Provow
+Prophete
+Procaccini
+Pritz
+Prillaman
+Priess
+Pretlow
+Prestia
+Presha
+Prescod
+Preast
+Praytor
+Prashad
+Praino
+Pozzi
+Pounder
+Pottenger
+Potash
+Porada
+Popplewell
+Ponzo
+Ponter
+Pommier
+Polland
+Polidori
+Polasky
+Pola
+Pok
+Poitier
+Poisso
+Poire
+Point
+Pofahl
+Podolsky
+Podell
+Plueger
+Plowe
+Plotz
+Plotnik
+Ploch
+Pliska
+Plessner
+Plaut
+Platzer
+Plake
+Pizzino
+Pizza
+Pirog
+Piquette
+Pipho
+Pioche
+Pintos
+Pinkert
+Pinet
+Pilkerton
+Pilch
+Pilarz
+Pignataro
+Piermatteo
+Picozzi
+Pickler
+Pickette
+Pichler
+Philogene
+Pheasant
+Phare
+Phang
+Pfrogner
+Pfisterer
+Pettinelli
+Petruzzi
+Petrovic
+Petretti
+Petermeier
+Pestone
+Pesterfield
+Pessin
+Pesch
+Persky
+Perruzza
+Perrott
+Perritt
+Perretti
+Perrera
+Peroutka
+Peroni
+Peron
+Peret
+Perdew
+Perazzo
+Peppe
+Peno
+Penberthy
+Penagos
+Peles
+Pelech
+Peiper
+Peight
+Pefferman
+Peddie
+Peckenpaugh
+Pean
+Payen
+Pavloski
+Pavlica
+Paullin
+Pattie
+Patteson
+Passon
+Passey
+Passe
+Passalacqua
+Pasquini
+Paskel
+Parter
+Partch
+Parriott
+Parrella
+Parraz
+Parmely
+Parizo
+Parisian
+Papelian
+Papasergi
+Pantojz
+Panto
+Panich
+Panchal
+Palys
+Palms
+Pallone
+Palinski
+Pali
+Palevic
+Pale
+Pagels
+Paciorek
+Pacho
+Pacella
+Paar
+Ozbun
+Overweg
+Overholser
+Ovalles
+Outhouse
+Outcalt
+Otterbein
+Otta
+Ostergren
+Osher
+Osbon
+Orzech
+Orwick
+Orrico
+Oropesa
+Orn
+Ormes
+Orillion
+Opal
+Onorati
+Onnen
+Omary
+Olk
+Olding
+Okonski
+Okimoto
+Ohlrich
+Ohayon
+Oguin
+Ogley
+Oftedahl
+Offen
+Ofallon
+Oeltjen
+Odam
+Ockmond
+Ockimey
+Ocean
+Obermeyer
+Oberdorf
+Obanner
+Oballe
+Oard
+Oakden
+Nyhan
+Nydam
+Numan
+Noyer
+Notte
+Nothstein
+Notestine
+Noser
+Nork
+Nolde
+Noa
+Nishihara
+Nishi
+Nikolic
+Nihart
+Nietupski
+Niesen
+Niehus
+Niece
+Nidiffer
+Nicoulin
+Nicolaysen
+Nicklow
+Nickl
+Nickeson
+Nichter
+Nicholl
+Ngyun
+Newsham
+Newmann
+Neveux
+Neuzil
+Neumayer
+Netland
+Nessen
+Nesheim
+Nelli
+Nelke
+Necochea
+Nazari
+Navy
+Navorro
+Navarez
+Navan
+Natter
+Natt
+Nater
+Nasta
+Narvaiz
+Nardelli
+Napp
+Nakahara
+Nairn
+Nagg
+Nager
+Nagano
+Nafziger
+Naffziger
+Nadelson
+Muzzillo
+Murri
+Murrey
+Murgia
+Murcia
+Muno
+Munier
+Mulqueen
+Mulliniks
+Mulkins
+Mulik
+Muhs
+Muffley
+Mozell
+Moynahan
+Mounger
+Mottley
+Motil
+Moseman
+Moseby
+Mosakowski
+Morten
+Mortell
+Morrisroe
+Morrero
+Mormino
+Morland
+Morger
+Morgenthaler
+Moren
+Morelle
+Morawski
+Morasca
+Morang
+Morand
+Moog
+Montney
+Montera
+Montee
+Montane
+Montagne
+Mons
+Monohan
+Monnett
+Monkhouse
+Moncure
+Momphard
+Molyneaux
+Molles
+Mollenkopf
+Molette
+Moland
+Mohs
+Mohmand
+Mohlke
+Moessner
+Moers
+Mockus
+Moccio
+Mlinar
+Mizzelle
+Mittler
+Mitri
+Mitchusson
+Mitchen
+Mistrot
+Mistler
+Misch
+Miriello
+Minkin
+Mininger
+Minerich
+Minehart
+Minderman
+Minden
+Minahan
+Milonas
+Millon
+Millholland
+Milleson
+Millerbernd
+Millage
+Militante
+Milionis
+Milhoan
+Mildenberger
+Milbury
+Mikolajczak
+Miklos
+Mikkola
+Mikes
+Migneault
+Mifsud
+Mietus
+Mieszala
+Mielnicki
+Midy
+Michon
+Michioka
+Micheau
+Michaeli
+Micali
+Methe
+Metallo
+Messler
+Mesch
+Merow
+Meroney
+Mergenthaler
+Meres
+Mercy
+Menuey
+Menousek
+Menning
+Menn
+Menghini
+Mendia
+Memmer
+Melot
+Mellow
+Mellenthin
+Melland
+Meland
+Meixner
+Meisenheimer
+Meineke
+Meinders
+Mehrens
+Mehlig
+Meglio
+Medsker
+Medicine
+Medero
+Mederios
+Meabon
+Mcwright
+Mcright
+Mcreath
+Mcrary
+Mcquirter
+Mcquerry
+Mcquary
+Mcphie
+Mcnurlen
+Mcnelley
+Mcnee
+Mcnairy
+Mcmanamy
+Mcmahen
+Mckowen
+Mckiver
+Mckinlay
+Mckearin
+Mcirvin
+Mcintrye
+Mchorse
+Mchaffie
+Mcgroarty
+Mcgoff
+Mcgivern
+Mceniry
+Mcelhiney
+Mcdiarmid
+Mccullars
+Mccubbins
+Mccrimon
+Mccovery
+Mccommons
+Mcclour
+Mccarrick
+Mccarey
+Mccallen
+Mcbrien
+Mcarthy
+Mayone
+Maybin
+Maximo
+Maxam
+Maurais
+Maughn
+Matzek
+Matts
+Matin
+Mathre
+Mathia
+Mateen
+Matava
+Masso
+Massar
+Massanet
+Masingale
+Mascaro
+Marthaler
+Martes
+Marso
+Marshman
+Marsalis
+Marrano
+Marolt
+Marold
+Markins
+Margulis
+Mardirosian
+Marchiano
+Marchak
+Marandola
+Marana
+Manues
+Mantis
+Mante
+Mansukhani
+Mansi
+Mannan
+Maniccia
+Mangine
+Manery
+Mandigo
+Manda
+Mancell
+Mamo
+Malstrom
+Malouf
+Malenfant
+Malena
+Maldenado
+Malandruccolo
+Malak
+Malabanan
+Makino
+Maj
+Maisonave
+Mainord
+Maino
+Mainard
+Maillard
+Maia
+Mahmud
+Mahdi
+Mahapatra
+Mahaley
+Mahaffy
+Magouirk
+Maglaras
+Magat
+Magan
+Maga
+Maffia
+Madrazo
+Madrano
+Maditz
+Mackert
+Mackellar
+Mackell
+Macht
+Macchia
+Maccarthy
+Maahs
+Lytal
+Lye
+Luzar
+Luzader
+Lutjen
+Lunger
+Lunan
+Luma
+Lukins
+Luhmann
+Luers
+Ludvigsen
+Ludlam
+Ludemann
+Luchini
+Lucente
+Lubrano
+Lubow
+Luber
+Lubeck
+Lowing
+Loven
+Loup
+Louise
+Louge
+Losco
+Lorts
+Lormand
+Lorenzetti
+Longford
+Longden
+Longbrake
+Lokhmatov
+Loge
+Loeven
+Loeser
+Locket
+Locey
+Locatelli
+Litka
+Lista
+Lisonbee
+Lisenbee
+Liscano
+Liranzo
+Liquori
+Liptrot
+Lionetti
+Lio
+Linscomb
+Linkovich
+Linington
+Lingefelt
+Lindler
+Lindig
+Lindall
+Lincks
+Linander
+Linan
+Limburg
+Limbrick
+Limbach
+Likos
+Lighthall
+Liford
+Lietzke
+Liebe
+Liddicoat
+Lickley
+Lichter
+Libel
+Lias
+Liapis
+Lezo
+Lewan
+Levitz
+Levesgue
+Leverson
+Levander
+Leuthauser
+Letbetter
+Lesuer
+Lesmeister
+Lesly
+Lerer
+Leppanen
+Lepinski
+Leota
+Lenherr
+Lembrick
+Lelonek
+Leisten
+Leiss
+Leins
+Leingang
+Leinberger
+Leinbach
+Leikam
+Leidig
+Lehtonen
+Lehnert
+Lehew
+Legier
+Lefchik
+Lecy
+Leconte
+Lecher
+Lebrecht
+Leather
+Leaper
+Lawter
+Lawrenz
+Lavy
+Laur
+Lauderbaugh
+Lauden
+Laudato
+Latting
+Latsko
+Latini
+Lassere
+Lasseigne
+Laspina
+Laso
+Laslie
+Laskowitz
+Laske
+Laser
+Lasenby
+Lascola
+Lariosa
+Larcade
+Lapete
+Laperouse
+Lanuza
+Lanting
+Lantagne
+Lansdale
+Lanphier
+Langmaid
+Langella
+Lanese
+Landrus
+Lampros
+Lamens
+Laizure
+Laitinen
+Laigle
+Lahm
+Lagueux
+Lagorio
+Lagomarsino
+Lagasca
+Lagana
+Lafont
+Laflen
+Lafavor
+Lafarge
+Laducer
+Ladnier
+Ladesma
+Lacognata
+Lackland
+Lacerte
+Labuff
+Laborin
+Labine
+Labauve
+Kuzio
+Kusterer
+Kussman
+Kusel
+Kusch
+Kurutz
+Kurdyla
+Kupka
+Kunzler
+Kunsman
+Kuni
+Kuney
+Kunc
+Kulish
+Kuliga
+Kulaga
+Kuilan
+Kuhre
+Kuhnke
+Kuemmerle
+Kueker
+Kudla
+Kudelka
+Kubinski
+Kubicki
+Kubal
+Krzyzanowski
+Krupicka
+Krumwiede
+Krumme
+Kross
+Kropidlowski
+Krokos
+Kroell
+Kritzer
+Kribs
+Kreitlow
+Kreisher
+Kraynak
+Krass
+Kranzler
+Kramb
+Kozyra
+Kozicki
+Kovalik
+Kovalchik
+Kovacevic
+Kotula
+Kotrba
+Koteles
+Kosowski
+Koskela
+Kosiba
+Koscinski
+Kosch
+Kory
+Korab
+Kopple
+Kopper
+Koppelman
+Koppel
+Konwinski
+Kon
+Kolosky
+Koloski
+Kolinsky
+Kolinski
+Kolbeck
+Kolasa
+Koepf
+Koda
+Kochevar
+Kochert
+Kobs
+Knust
+Knueppel
+Knoy
+Knieriem
+Knier
+Kneller
+Knappert
+Klitz
+Klintworth
+Klinkenberg
+Klinck
+Kleindienst
+Kleeb
+Klecker
+Kjellberg
+Kitten
+Kitsmiller
+Kisor
+Kisiel
+Kise
+Kirbo
+Kio
+Kinzle
+Kinkaid
+Kingsford
+Kingry
+Kimpton
+Kimel
+Kimberley
+Killmon
+Killick
+Kilgallon
+Kilcher
+Kihn
+Kiggins
+Kiecker
+Kher
+Khaleel
+Keziah
+Kettell
+Ketchen
+Keshishian
+Kersting
+Kersch
+Kerins
+Kercher
+Keno
+Kenefick
+Kemph
+Kempa
+Kelsheimer
+Kelln
+Kellenberger
+Kekahuna
+Keisling
+Keirnan
+Keimig
+Kehn
+Keal
+Ke
+Kaupp
+Kaufhold
+Kauffmann
+Katzenberg
+Katona
+Kaszynski
+Kaszuba
+Kassebaum
+Kasa
+Kartye
+Kartchner
+Karstens
+Karpinsky
+Karmely
+Karel
+Karasek
+Kapral
+Kaper
+Kanelos
+Kanahele
+Kampmann
+Kampe
+Kalp
+Kallus
+Kallevig
+Kallen
+Kaliszewski
+Kaleohano
+Kalchthaler
+Kalama
+Kalahiki
+Kaili
+Kahawai
+Kagey
+Justiss
+Jurkowski
+Jurgensmeyer
+Juilfs
+Josue
+Jopling
+Jondahl
+Jomes
+Joice
+Johannessen
+Joeckel
+Jezewski
+Jezek
+Jeswald
+Jervey
+Jeppsen
+Jenniges
+Jennifer
+Jennett
+Jemmott
+Jeffs
+Jeffry
+Jaurequi
+Janisch
+Janick
+Janice
+Jacek
+Jacaruso
+Iwanicki
+Ishihara
+Isenberger
+Isbister
+Iruegas
+Inzer
+Inyart
+Inscore
+Innocenti
+Inglish
+Infantolino
+Indovina
+Inaba
+Imondi
+Imdieke
+Imbert
+Illes
+Ida
+Iarocci
+Iannucci
+Huver
+Hutley
+Husser
+Husmann
+Hupf
+Huntsberger
+Hunnewell
+Hullum
+Huit
+Huish
+Huh
+Hughson
+Huft
+Hufstetler
+Hueser
+Hudnell
+Hovden
+Housen
+Houghtling
+Hoth
+Hossack
+Hoshaw
+Horsford
+Horry
+Hornbacher
+Horde
+Hoppenstedt
+Hopkinson
+Honza
+Honor
+Homann
+Holzmeister
+Holycross
+Holverson
+Holtzlander
+Holroyd
+Holmlund
+Hollywood
+Holderness
+Holderfield
+Holck
+Hojnacki
+Hohlfeld
+Hohenberger
+Hoganson
+Hogancamp
+Hoffses
+Hoerauf
+Hoell
+Hoefert
+Hodum
+Hoder
+Hockenbury
+Hoage
+Hisserich
+Hislip
+Hirons
+Hippensteel
+Hippen
+Hinkston
+Hindes
+Hinchcliff
+Hin
+Himmel
+Hillberry
+Hildring
+Hiester
+Hiefnar
+Hides
+Hibberd
+Hibben
+Heyliger
+Heyl
+Heyes
+Hevia
+Heu
+Hettrick
+Hert
+Hersha
+Hernandz
+Herkel
+Herber
+Henscheid
+Hennesy
+Henly
+Henegan
+Henebry
+Hench
+Hemsath
+Hemm
+Hemken
+Hemann
+Heltzel
+Hellriegel
+Hejny
+Heinl
+Heinke
+Heidinger
+Hegeman
+Hefferan
+Hedglin
+Hebdon
+Hearnen
+Hearing
+Heape
+Heagy
+Headings
+Headd
+Hazelbaker
+Havlick
+Hauschildt
+Haury
+Hassenfritz
+Hasenbeck
+Haseltine
+Hartstein
+Hartry
+Hartnell
+Harston
+Harpool
+Harmen
+Hardister
+Hardey
+Harders
+Harbolt
+Harbinson
+Haraway
+Haque
+Hansmann
+Hanser
+Hansch
+Hansberry
+Hankel
+Hanigan
+Haneline
+Hampe
+Hamons
+Hammerstone
+Hammerle
+Hamme
+Hammargren
+Hamelton
+Hamberger
+Hamasaki
+Halprin
+Halman
+Hallihan
+Halen
+Haldane
+Hails
+Haifley
+Hai
+Hages
+Hagadorn
+Hadwin
+Habicht
+Habermehl
+Gyles
+Gutzman
+Gutekunst
+Gustason
+Gusewelle
+Gurnsey
+Gurnee
+Gunterman
+Gumina
+Gulliver
+Gulbrandson
+Guiterez
+Guerino
+Guedry
+Gucwa
+Guardarrama
+Guagliano
+Guadagno
+Grulke
+Groote
+Groody
+Groft
+Groeneweg
+Grochow
+Grippe
+Grimstead
+Griepentrog
+Greenfeld
+Greenaway
+Grebe
+Graziosi
+Graw
+Gravina
+Grassie
+Grapes
+Granzow
+Grandjean
+Granby
+Gramacy
+Graces
+Gozalez
+Goyer
+Gotch
+Gosden
+Gorny
+Gormont
+Goodness
+Goodgion
+Gonya
+Gonnerman
+Gompert
+Golish
+Goligoski
+Goldmann
+Goike
+Goetze
+Godeaux
+Glenna
+Glaza
+Glassel
+Glaspy
+Glander
+Glady
+Giumarro
+Gitelman
+Gisondi
+Gismondi
+Girvan
+Girten
+Gironda
+Giovinco
+Ginkel
+Gilster
+Giesy
+Gierman
+Giddins
+Giardini
+Gianino
+Ghea
+Geurin
+Gett
+Getson
+Gerrero
+Germond
+Gere
+Gentsy
+Genta
+Gennette
+Genito
+Genis
+Gene
+Gendler
+Geltz
+Geiss
+Gehret
+Gegenheimer
+Geffert
+Geeting
+Gebel
+Gavette
+Gavenda
+Gaumond
+Gaudioso
+Gatzke
+Gatza
+Gattshall
+Gaton
+Gatchel
+Gasperi
+Gaska
+Gasiorowski
+Garritson
+Garrigus
+Garnier
+Garnick
+Gardinier
+Gardenas
+Garcy
+Garate
+Gandolfi
+Gamm
+Gamel
+Gambel
+Gallmon
+Gallemore
+Gallati
+Gainous
+Gainforth
+Gahring
+Gaffey
+Gaebler
+Gadzinski
+Gadbury
+Gabri
+Gabe
+Gaba
+Fyke
+Furtaw
+Furnas
+Furcron
+Funn
+Funck
+Fulwood
+Fulvio
+Fullmore
+Fukumoto
+Fuest
+Fuery
+Fuente
+Fuel
+Frymire
+Frush
+Frohlich
+Froedge
+Frodge
+Fritzinger
+Fricker
+Frericks
+Frein
+Freid
+Freggiaro
+Fratto
+Franzi
+Franciscus
+Fralix
+Fowble
+Fotheringham
+Foslien
+Foshie
+Fortmann
+Forsey
+Forkner
+Foppiano
+Fontanetta
+Fonohema
+Fogler
+Fockler
+Fluty
+Flusche
+Flud
+Florin
+Flori
+Flenory
+Fleharty
+Fleeks
+Flaxman
+Flash
+Flaming
+Fiumara
+Fitzmorris
+Finnicum
+Finkley
+Fineran
+Fillhart
+Filipi
+Fijal
+Fieldson
+Ficken
+Ficarra
+Fetch
+Festerman
+Fess
+Ferryman
+Ferner
+Fergason
+Ferell
+Fennern
+Femmer
+Feldmeier
+Feeser
+Feenan
+Federick
+Fedak
+Febbo
+Feazell
+Fearing
+Fazzone
+Fauth
+Fauset
+Faurote
+Faulker
+Faubion
+Fatzinger
+Fasick
+Fanguy
+Fambrough
+Falks
+Fahl
+Fabio
+Faaita
+Exler
+Ewens
+Estrado
+Esten
+Esteen
+Esquivez
+Espejo
+Esmiol
+Esguerra
+Esco
+Ertz
+Erspamer
+Ernstes
+Erisman
+Erhard
+Ereaux
+Ercanbrack
+Erbes
+Epple
+Entsminger
+Entriken
+Enslow
+Ennett
+Engquist
+Englebert
+Englander
+Engesser
+Engert
+Engeman
+Enge
+Enerson
+End
+Emhoff
+Emge
+Emerald
+Elting
+Ellner
+Ellenberg
+Ellenbecker
+Elio
+Elfert
+Elden
+Elawar
+Ekstrand
+Eison
+Eismont
+Eisenbrandt
+Eiseman
+Eischens
+Ehrgott
+Egley
+Egert
+Eddlemon
+Economy
+Eckerson
+Eckersley
+Eckberg
+Echeverry
+Eberts
+Earthman
+Earnhart
+Eapen
+Eachus
+Dykas
+Dust
+Dusi
+Durning
+During
+Durdan
+Dunomes
+Duncombe
+Dume
+Dullen
+Dullea
+Dulay
+Dul
+Duffett
+Dubs
+Dubard
+Drook
+Drenth
+Drahos
+Dragone
+Downin
+Downham
+Dowis
+Dowhower
+Doward
+Dovalina
+Dost
+Dopazo
+Doose
+Donson
+Donnan
+Dominski
+Dollarhide
+Dolinar
+Dolecki
+Dolbee
+Doege
+Dockus
+Dobler
+Dobkin
+Dobias
+Divoll
+Diviney
+Ditter
+Ditman
+Dissinger
+Dismang
+Dirlam
+Dinneen
+Dini
+Dingwall
+Dine
+Din
+Diloreto
+Dilmore
+Dillaman
+Dikeman
+Diiorio
+Dighton
+Diffley
+Dieudonne
+Dietel
+Dieringer
+Diercks
+Dienhart
+Diekrager
+Diefendorf
+Dicke
+Dicamillo
+Dibrito
+Dibona
+Dezeeuw
+Dewhurst
+Devins
+Deviney
+Deupree
+Detherage
+Despino
+Desmith
+Desjarlais
+Deshner
+Desha
+Desanctis
+Derring
+Derousse
+Derobertis
+Deridder
+Derego
+Derden
+Deprospero
+Deprofio
+Depping
+Deperro
+Denty
+Denoncourt
+Dencklau
+Demler
+Demirchyan
+Demichiel
+Demesa
+Demere
+Demaggio
+Delung
+Deluise
+Delmoral
+Delmastro
+Delmas
+Delligatti
+Delle
+Delena
+Delasbour
+Delarme
+Delargy
+Delagrange
+Delafontaine
+Deist
+Deiss
+Deighan
+Dehoff
+Degrazia
+Degman
+Defosses
+Deforrest
+Deeks
+Decoux
+Decarolis
+Debuhr
+Deberg
+Debarr
+Debari
+Dearmon
+Deare
+Deardurff
+Daywalt
+Dayer
+Davoren
+Davignon
+Daviau
+Dauteuil
+Dauterive
+Daul
+Darnley
+Darlin
+Darakjy
+Dapice
+Dannunzio
+Danison
+Daniello
+Damario
+Dalonzo
+Dallis
+Daleske
+Dalenberg
+Daiz
+Dains
+Daines
+Dagnese
+Dady
+Dadey
+Czyzewski
+Czapor
+Czaplewski
+Czajka
+Cyganiewicz
+Cuttino
+Cutrona
+Cussins
+Cusanelli
+Cuperus
+Cundy
+Cumiskey
+Cumins
+Cuizon
+Cuffia
+Cuffe
+Cuffari
+Cuccaro
+Cubie
+Cryder
+Cruson
+Crounse
+Cromedy
+Cring
+Creer
+Credeur
+Crea
+Cozort
+Cozine
+Cowee
+Cowdery
+Coventry
+Couser
+Courtway
+Courington
+Cotman
+Costlow
+Costell
+Corton
+Corsaro
+Corrieri
+Corrick
+Corradini
+Coron
+Coren
+Cord
+Corbi
+Corado
+Copus
+Coppenger
+Cooperwood
+Coontz
+Coonce
+Contrera
+Connealy
+Conell
+Comtois
+Compere
+Commins
+Commings
+Comegys
+Coma
+Colyar
+Colo
+Collister
+Collick
+Collella
+Coler
+Colborn
+Cohran
+Cogbill
+Coffen
+Cocuzzo
+Clynes
+Closter
+Clock
+Clipp
+Clingingsmith
+Clemence
+Clayman
+Classon
+Clas
+Clarey
+Clarence
+Clague
+Ciubal
+Citrino
+Citarella
+Cirone
+Cipponeri
+Cindrich
+Cimo
+Ciliberto
+Cichowski
+Ciccarello
+Cicala
+Chura
+Chubbuck
+Chronis
+Christlieb
+Chriss
+Chizek
+Chittester
+Chiquito
+Chimento
+Childree
+Chianese
+Chevrette
+Cheese
+Checo
+Chastang
+Chargualaf
+Chapmon
+Chantry
+Chahal
+Chafetz
+Cezar
+Ceruantes
+Cerrillo
+Cerrano
+Cerecedes
+Cerami
+Cegielski
+Cavallero
+Catinella
+Cassata
+Caslin
+Casano
+Casacchia
+Caruth
+Cartrette
+Carten
+Carodine
+Carnrike
+Carnall
+Carmicle
+Carlan
+Carlacci
+Caris
+Cariaga
+Cardine
+Cardimino
+Cardani
+Carbonara
+Carano
+Capua
+Capponi
+Cappellano
+Caporale
+Capelli
+Canupp
+Cantrel
+Cantone
+Canterberry
+Cannizzo
+Cannan
+Canelo
+Caneer
+Candill
+Candee
+Campbel
+Caminero
+Camble
+Caluya
+Callicott
+Calk
+Caito
+Caffie
+Caden
+Cadavid
+Cacy
+Cachu
+Cachola
+Cabreja
+Cabiles
+Cabada
+Caamano
+Byran
+Byon
+Buyck
+Bussman
+Bussie
+Bushner
+Burston
+Burnison
+Burkman
+Burkhammer
+Bures
+Burdeshaw
+Bumpass
+Bullinger
+Bullers
+Bulgrin
+Bugay
+Buffalo
+Budak
+Buczynski
+Buckendorf
+Buccieri
+Bubrig
+Brynteson
+Brunz
+Brunmeier
+Brunkow
+Brunetto
+Brunelli
+Brumwell
+Bruggman
+Brucki
+Brucculeri
+Brozovich
+Browing
+Brotman
+Broda
+Brocker
+Broadstreet
+Brix
+Britson
+Brinck
+Brimmage
+Brightly
+Brierre
+Bridenstine
+Brezenski
+Brezee
+Brevik
+Brest
+Brentlinger
+Brentley
+Breidenbach
+Breckel
+Brech
+Breaker
+Brazzle
+Braughton
+Brauch
+Brattin
+Brattain
+Branhan
+Branford
+Braner
+Brander
+Braly
+Braegelmann
+Brabec
+Boyt
+Boyack
+Bowren
+Bowl
+Bovian
+Boughan
+Botton
+Botner
+Bosques
+Borzea
+Borre
+Boron
+Bornhorst
+Borgstrom
+Borella
+Boop
+Bontempo
+Bonniwell
+Bonnes
+Bonjour
+Bonillo
+Bonano
+Bolek
+Bohol
+Bohaty
+Boffa
+Boetcher
+Boesen
+Boepple
+Boehler
+Boedecker
+Boeckx
+Bodi
+Boal
+Bloodsworth
+Bloodgood
+Blome
+Blockett
+Blixt
+Blanchett
+Blackhurst
+Blackaby
+Bjornberg
+Bitzer
+Bittenbender
+Bitler
+Birchall
+Binnicker
+Binggeli
+Billett
+Bilberry
+Bijou
+Biglow
+Bierly
+Bielby
+Biegel
+Beu
+Berzas
+Berte
+Bertagnolli
+Berreth
+Bernhart
+Bergum
+Berentson
+Berenson
+Berdy
+Bercegeay
+Bentle
+Bentivegna
+Bentham
+Benscoter
+Benns
+Bennick
+Benjamine
+Beneze
+Benett
+Beneke
+Bendure
+Bendix
+Bendick
+Benauides
+Belman
+Bellus
+Bellott
+Bellefleur
+Bellas
+Beljan
+Belgard
+Beith
+Beinlich
+Beierle
+Behme
+Beevers
+Beermann
+Beeching
+Bedward
+Bedrosian
+Bedner
+Bedeker
+Bechel
+Becera
+Beaubrun
+Beardmore
+Bealmear
+Bazin
+Bazer
+Baumhoer
+Baumgarner
+Bauknecht
+Battson
+Battiest
+Basulto
+Baster
+Basques
+Basista
+Basiliere
+Bashi
+Barzey
+Barz
+Bartus
+Bartucca
+Bartek
+Barrero
+Barreca
+Barnoski
+Barndt
+Barklow
+Baribeau
+Barette
+Bares
+Barentine
+Bareilles
+Barch
+Barbre
+Barberi
+Barbagelata
+Baraw
+Baratto
+Baranoski
+Bar
+Baptise
+Bankson
+Bankey
+Bankard
+Banik
+Baltzley
+Ballen
+Balkey
+Balius
+Balderston
+Bakula
+Bakalar
+Baffuto
+Baerga
+Badoni
+Backous
+Bachtel
+Bachrach
+Baccari
+Babine
+Babilonia
+Baar
+Azbill
+Azad
+Aycox
+Ayalla
+Avolio
+Austerberry
+Aughtry
+Aufderheide
+Auch
+Attanasio
+Athayde
+Atcher
+Astor
+Asselta
+Aslin
+Aslam
+Ashwood
+Ashraf
+Ashbacher
+Asbridge
+Asakura
+Arzaga
+Arriaza
+Arrez
+Arrequin
+Arrants
+Armiger
+Armenteros
+Armbrister
+Arko
+Argumedo
+Arguijo
+Ardolino
+Arcia
+Arbizo
+Aravjo
+Aper
+Anzaldo
+Antu
+Antrikin
+Antony
+Antonia
+Antonetty
+Antinoro
+Anthon
+Antenucci
+Anstead
+Annese
+Ankrum
+Andreason
+Andrado
+Andaverde
+Anastos
+Anable
+Amsterdam
+Amspoker
+Amrine
+Amrein
+Amorin
+Amel
+Ambrosini
+Amber
+Alsbrook
+Alnutt
+Almasi
+Allessio
+Allateef
+Alison
+Aldous
+Alderink
+Aldaz
+Akmal
+Akard
+Aiton
+Aites
+Ainscough
+Aikey
+Ahrends
+Ahlm
+Aguada
+Agans
+Adelmann
+Adebisi
+Addesso
+Adaway
+Adamaitis
+Ackison
+Abud
+Abendroth
+Abdur
+Abdool
+Aamodt
+Zywiec
+Zwiefelhofer
+Zwahlen
+Zunino
+Zuehl
+Zmuda
+Zmolek
+Zizza
+Ziska
+Zinser
+Zinkievich
+Zinger
+Zingarelli
+Ziesmer
+Ziegenfuss
+Ziebol
+Zettlemoyer
+Zettel
+Zervos
+Zenke
+Zembower
+Zelechowski
+Zelasko
+Zeise
+Zeek
+Zeeb
+Zarlenga
+Zarek
+Zaidi
+Zahnow
+Zahnke
+Zaharis
+Zach
+Zacate
+Zabrocki
+Zaborac
+Yurchak
+Yuengling
+Younie
+Youngers
+Youell
+Yott
+Yoshino
+Yorks
+Yordy
+Yochem
+Yerico
+Yerdon
+Yeiser
+Yearous
+Yearick
+Yeaney
+Ybarro
+Yasutake
+Yasin
+Yanke
+Yanish
+Yanik
+Yamazaki
+Yamat
+Yaggi
+Ximenez
+Wyzard
+Wynder
+Wyly
+Wykle
+Wutzke
+Wuori
+Wuertz
+Wuebker
+Wrightsel
+Worobel
+Worlie
+Worford
+Worek
+Woolson
+Woodrome
+Woodly
+Woodling
+Wontor
+Wondra
+Woltemath
+Wollmer
+Wolinski
+Wolfert
+Wojtanik
+Wojtak
+Wohlfarth
+Woeste
+Wobbleton
+Witz
+Wittmeyer
+Witchey
+Wisotzkey
+Wisnewski
+Wisman
+Wirch
+Wippert
+Wineberg
+Wimpee
+Wilusz
+Wiltsey
+Willig
+Williar
+Willers
+Willadsen
+Wilfred
+Wildhaber
+Wilday
+Wigham
+Wiggen
+Wiewel
+Wieting
+Wietbrock
+Wiesel
+Wiesehan
+Wiersema
+Wiegert
+Widney
+Widmark
+Wickson
+Wickings
+Wichern
+Whtie
+Whittie
+Whitlinger
+Whitfill
+Whitebread
+Whispell
+Whetten
+Wheeley
+Wheeles
+Wheelen
+Whatcott
+Weyland
+Weter
+Westrup
+Westphalen
+Westly
+Westland
+Wessler
+Wesolick
+Wesler
+Wesche
+Werry
+Wero
+Wernecke
+Werkhoven
+Wellspeak
+Wellings
+Welford
+Welander
+Weissgerber
+Weisheit
+Weins
+Weill
+Weigner
+Wehrmann
+Wehrley
+Wehmeier
+Wege
+Weers
+Weavers
+Watring
+Wassum
+Wassman
+Wassil
+Washabaugh
+Wascher
+Wary
+Warth
+Warbington
+Wanca
+Wammack
+Wamboldt
+Walterman
+Walkington
+Walkenhorst
+Walinski
+Wakley
+Wagg
+Wadell
+Vuckovich
+Voogd
+Voller
+Vokes
+Vogle
+Vogelsberg
+Vodicka
+Vissering
+Visage
+Vipond
+Vincik
+Villalona
+Vil
+Vickerman
+Vettel
+Veteto
+Vessel
+Vesperman
+Vesco
+Vertucci
+Versaw
+Verba
+Ventris
+Venecia
+Vendela
+Venanzi
+Veldhuizen
+Vehrs
+Veer
+Vee
+Vay
+Vaughen
+Vasilopoulos
+Vascocu
+Varvel
+Varno
+Varlas
+Varland
+Vario
+Vareschi
+Vanwyhe
+Vanweelden
+Vansciver
+Vannaman
+Vanluven
+Vanloo
+Vanlaningham
+Vankomen
+Vanhout
+Vanhampler
+Vangorp
+Vangorden
+Vanella
+Vandresar
+Vandis
+Vandeyacht
+Vandewerker
+Vandevsen
+Vanderwall
+Vandercook
+Vanderberg
+Vanbergen
+Valko
+Valesquez
+Valeriano
+Valen
+Vachula
+Vacha
+Uzee
+Uva
+Uselman
+Urizar
+Urion
+Urben
+Upthegrove
+Unzicker
+Unsell
+Unick
+Umscheid
+Umin
+Umanzor
+Ullo
+Ulicki
+Uhlir
+Uddin
+Tytler
+Tymeson
+Tyger
+Twisdale
+Twedell
+Tweddle
+Turrey
+Tures
+Turell
+Tur
+Tupa
+Tuitt
+Tuberville
+Tubby
+Tryner
+Trumpower
+Trumbore
+Truly
+Troglen
+Troff
+Troesch
+Trivisonno
+Tritto
+Tritten
+Tritle
+Trippany
+Tringali
+Tretheway
+Treon
+Trench
+Trejos
+Tregoning
+Treffert
+Traycheff
+Travali
+Trauth
+Trauernicht
+Transou
+Trane
+Trana
+Toves
+Tosta
+Torp
+Tornquist
+Tornes
+Torchio
+Toppings
+Toor
+Tooks
+Tonks
+Tomblinson
+Tomala
+Tollinchi
+Tolles
+Tokich
+Toh
+Tofte
+Todman
+Toddy
+Titze
+Timpone
+Tillema
+Tier
+Tienken
+Tiblier
+Thyberg
+Thursby
+Thurrell
+Thurm
+Thruman
+Thorsted
+Thorley
+Thomer
+Thoen
+Thissen
+Theimer
+Thee
+Thayn
+Thanpaeng
+Thammavongsa
+Thalman
+Texiera
+Texidor
+Teverbaugh
+Teska
+Ternullo
+Teplica
+Tepe
+Teno
+Tenholder
+Tenbusch
+Tenbrink
+Temby
+Tejedor
+Teitsworth
+Teichmann
+Tehan
+Tegtmeyer
+Tees
+Teem
+Tays
+Taubert
+Tauares
+Taschler
+Tartamella
+Tarquinio
+Tarbutton
+Tappendorf
+Tapija
+Tansil
+Tannahill
+Tamondong
+Talahytewa
+Takashima
+Taecker
+Tabora
+Tabin
+Tabbert
+Szymkowski
+Szymanowski
+Syversen
+Syrett
+Syracuse
+Synnott
+Sydnes
+Swimm
+Sweney
+Swearegene
+Swartzel
+Swanstrom
+Svedin
+Suss
+Suryan
+Surrey
+Supplice
+Supnet
+Suoboda
+Sundby
+Sumaya
+Sumabat
+Sulzen
+Sukovaty
+Sukhu
+Sugerman
+Sugalski
+Sugai
+Sudweeks
+Sudbeck
+Sucharski
+Stutheit
+Stumfoll
+Stuffle
+Struyk
+Strutz
+Strumpf
+Strowbridge
+Strothman
+Strojny
+Strohschein
+Stroffolino
+Stribble
+Strevel
+Strenke
+Stremming
+Strehle
+Strattman
+Stranak
+Stram
+Stracke
+Stoudamire
+Storks
+Stopp
+Stonebreaker
+Stolt
+Stoica
+Stofer
+Stockham
+Stockfisch
+Stjuste
+Stiteler
+Stiman
+Stillions
+Stillabower
+Stierle
+Sterlace
+Sterk
+Stepps
+Stenquist
+Stenner
+Stellman
+Steines
+Steinbaugh
+Steinbacher
+Steiling
+Steidel
+Steffee
+Stavinoha
+Staver
+Stastny
+Stasiuk
+Starrick
+Starliper
+Starlin
+Staniford
+Staner
+Standre
+Standefer
+Standafer
+Stanczyk
+Stallsmith
+Stagliano
+Staehle
+Staebler
+Stady
+Stadtmiller
+Squyres
+Spurbeck
+Sprunk
+Spranger
+Spoonamore
+Spoden
+Spilde
+Spezio
+Speros
+Sperandio
+Specchio
+Spearin
+Spayer
+Spallina
+Spadafino
+Sovie
+Sotello
+Sortor
+Sortino
+Sorrow
+Soros
+Sorola
+Sorbello
+Sonner
+Sonday
+Somes
+Soloway
+Soledad
+Soens
+Soellner
+Soderblom
+Sobin
+Sniezek
+Sneary
+Smyly
+Smutnick
+Smoots
+Smoldt
+Smitz
+Smitreski
+Smallen
+Smades
+Slunaker
+Sluka
+Slown
+Slovick
+Slocomb
+Slinger
+Slife
+Slicker
+Sleeter
+Slanker
+Skufca
+Skubis
+Skrocki
+Skov
+Skjei
+Skilton
+Skill
+Skarke
+Skalka
+Skalak
+Skaff
+Sixkiller
+Sitze
+Siter
+Sisko
+Sirman
+Sirls
+Sinotte
+Sinon
+Sincock
+Sincebaugh
+Simmoms
+Similien
+Silvius
+Silton
+Silloway
+Sikkema
+Sieracki
+Sienko
+Siemon
+Siemer
+Siefker
+Sieberg
+Siebens
+Siebe
+Sicurella
+Sicola
+Sickle
+Shumock
+Shumiloff
+Shuffstall
+Shuemaker
+Shuart
+Shu
+Shroff
+Shreeve
+Shostak
+Shortes
+Shorr
+Shivley
+Shintaku
+Shindo
+Shimomura
+Shiigi
+Sherow
+Sherburn
+Shepps
+Shenefield
+Shelvin
+Shelstad
+Shelp
+Sheild
+Sheaman
+Shaulis
+Sharrer
+Sharps
+Sharpes
+Shareef
+Shappy
+Shapero
+Shanor
+Shandy
+Shad
+Seyller
+Severn
+Sessom
+Sesley
+Servidio
+Serrin
+Sero
+Serge
+Septon
+Septer
+Sennott
+Sengstock
+Senff
+Senese
+Semprini
+Semone
+Sembrat
+Selva
+Sella
+Selbig
+Seiner
+Seif
+Seidt
+Sehrt
+Seemann
+Seelbinder
+Sedlay
+Sebert
+Searing
+Seaholm
+Seacord
+Seaburg
+Se
+Scungio
+Scroggie
+Scritchfield
+Scripture
+Scrimpsher
+Scrabeck
+Score
+Scorca
+Scobey
+Scivally
+Schwulst
+Schwinn
+Schwieson
+Schwery
+Schweppe
+Schwartzenbur
+Schurz
+Schumm
+Schulenburg
+Schuff
+Schuerholz
+Schryer
+Schrager
+Schorsch
+Schonhardt
+Schoenfelder
+Schoeck
+Schoeb
+Schnitzler
+Schnick
+Schnautz
+Schmig
+Schmelter
+Schmeichel
+Schluneger
+Schlosberg
+Schlobohm
+Schlenz
+Schlembach
+Schleisman
+Schleining
+Schleiff
+Schleider
+Schink
+Schilz
+Schiffler
+Schiavi
+Scheuer
+Schemonia
+Scheman
+Schelb
+Schaul
+Schaufelberge
+Scharer
+Schardt
+Scharbach
+Schabacker
+Scee
+Scavone
+Scarth
+Scarfone
+Scalese
+Sayne
+Sayed
+Savitz
+Satterlund
+Sattazahn
+Satow
+Sastre
+Sarr
+Sarjeant
+Sarff
+Sardella
+Santoya
+Santoni
+Santai
+Sankowski
+Sanft
+Sandow
+Sandoe
+Sandhaus
+Sandefer
+Sampey
+Samperi
+Sammarco
+Samia
+Samek
+Samay
+Samaan
+Salvadore
+Saltness
+Salsgiver
+Saller
+Salaz
+Salano
+Sakal
+Saka
+Saintlouis
+Saile
+Sahota
+Saggese
+Sagastume
+Sagan
+Sadri
+Sadak
+Sachez
+Saalfrank
+Saal
+Saadeh
+Ryu
+Rynn
+Ryley
+Ryle
+Rygg
+Rybarczyk
+Ruzich
+Ruyter
+Ruvo
+Rupel
+Ruopp
+Rundlett
+Runde
+Rundall
+Runck
+Rukavina
+Ruggiano
+Rufi
+Ruef
+Rubright
+Rubbo
+Rowbottom
+Route
+Rotner
+Rotman
+Rothweiler
+Rothlisberger
+Rosseau
+Rossean
+Rossa
+Roso
+Rosiek
+Roshia
+Rosenkrans
+Rosener
+Rosencrantz
+Rosencrans
+Rosello
+Roques
+Rookstool
+Rondo
+Romasanta
+Romack
+Rokus
+Rohweder
+Rog
+Roethler
+Roediger
+Rodwell
+Rodrigus
+Rodenbeck
+Rodefer
+Rodarmel
+Rockman
+Rockholt
+Rockford
+Rochow
+Roches
+Roblin
+Roblez
+Roble
+Robers
+Roat
+Rizza
+Rizvi
+Rizk
+Rixie
+Riveiro
+Rius
+Ritschard
+Ritrovato
+Risi
+Rishe
+Rippon
+Rinks
+Rings
+Ringley
+Ringgenberg
+Ringeisen
+Rimando
+Rilley
+Rijos
+Rieks
+Rieken
+Riechman
+Riddley
+Ricord
+Rickabaugh
+Richmeier
+Richesin
+Reyolds
+Rexach
+Revere
+Requena
+Reppucci
+Reposa
+Renzulli
+Renter
+Renault
+Remondini
+Relic
+Reither
+Reisig
+Reifsnider
+Reifer
+Reibsome
+Reibert
+Rehor
+Rehmann
+Reedus
+Redshaw
+Redfox
+Reczek
+Recupero
+Recor
+Reckard
+Recher
+Rear
+Realbuto
+Razer
+Rayman
+Raycraft
+Rayas
+Rawle
+Raviscioni
+Ravetto
+Ravenelle
+Rauth
+Raup
+Rattliff
+Rattley
+Rathfon
+Rataj
+Rasnic
+Rappleyea
+Rapaport
+Ransford
+Rann
+Rampersad
+Ramis
+Ramcharan
+Rainha
+Rainforth
+Ragans
+Ragains
+Rafidi
+Raffety
+Raducha
+Radsky
+Radler
+Radatz
+Raczkowski
+Rack
+Rabenold
+Quraishi
+Quinerly
+Quiet
+Quercia
+Quarnstrom
+Qian
+Pusser
+Puppo
+Pullan
+Pulis
+Pugel
+Puccini
+Puca
+Pruna
+Prowant
+Provines
+Pronk
+Prinkleton
+Prindall
+Primas
+Priesmeyer
+Pridgett
+Prevento
+Preti
+Presser
+Presnall
+Preseren
+Presas
+Presa
+Prchal
+Prattis
+Pratillo
+Praska
+Prak
+Powis
+Powderly
+Postlewait
+Postle
+Posch
+Porteus
+Portal
+Porraz
+Popwell
+Popoff
+Poplaski
+Poniatoski
+Pollina
+Polle
+Polhill
+Poletti
+Polaski
+Pokorney
+Poke
+Pointdexter
+Poinsette
+Po
+Ploszaj
+Plitt
+Pletz
+Pletsch
+Plemel
+Pleitez
+Playford
+Plaxco
+Platek
+Plambeck
+Plagens
+Placido
+Pisarski
+Pinuelas
+Pinnette
+Pinick
+Pinell
+Pinciaro
+Pinal
+Pilz
+Piltz
+Pillion
+Pilkinton
+Pilar
+Pikul
+Piepenburg
+Piening
+Piehler
+Piedrahita
+Piechocki
+Picknell
+Picker
+Pickelsimer
+Pich
+Picariello
+Phoeuk
+Phillipson
+Philbert
+Pherigo
+Phelka
+Peverini
+Petronis
+Petrina
+Petrash
+Petramale
+Petraglia
+Pery
+Personius
+Perrington
+Perrill
+Perpall
+Perot
+Perman
+Peragine
+Pentland
+Pennycuff
+Penninger
+Pennie
+Pennachio
+Penhall
+Pendexter
+Pencil
+Penalver
+Pelzel
+Pelter
+Pelow
+Pelo
+Peli
+Peinado
+Pedley
+Pecue
+Pecore
+Pechar
+Peairs
+Paynes
+Payano
+Pawelk
+Pavlock
+Pavlich
+Pavich
+Pavek
+Pautler
+Paulik
+Patmore
+Patella
+Patee
+Patalano
+Passini
+Passeri
+Paskell
+Parrigan
+Parmar
+Parayno
+Paparelli
+Pantuso
+Pante
+Panico
+Panduro
+Panagos
+Pama
+Palmo
+Pallotta
+Paling
+Palamino
+Pake
+Pajtas
+Pailthorpe
+Pahler
+Pagon
+Paglinawan
+Pagley
+Paget
+Paetz
+Paet
+Padley
+Pacleb
+Pacific
+Pachelo
+Pacer
+Paccione
+Pabey
+Ozley
+Ozimek
+Ozawa
+Owney
+Outram
+Oun
+Ouillette
+Oudekerk
+Ouch
+Ostrosky
+Ostermiller
+Ostermann
+Osterloh
+Osterfeld
+Ossenfort
+Osoria
+Oshell
+Orsino
+Orscheln
+Orrison
+Ororke
+Orf
+Orellano
+Orejuela
+Ordoyne
+Opsahl
+Opland
+Onofre
+Onaga
+Omahony
+Olszowka
+Olshan
+Ollig
+Oliff
+Olien
+Olexy
+Oldridge
+Oldfather
+Older
+Olalde
+Okun
+Okumoto
+Oktavec
+Okin
+Oka
+Ohme
+Ohlemacher
+Ohanesian
+Odneal
+Odgers
+Oderkirk
+Odden
+Ocain
+Obradovich
+Oakey
+Nussey
+Nunziato
+Nunoz
+Nunnenkamp
+Nuncio
+Noviello
+Novacek
+Nothstine
+Nostrand
+Northum
+Norsen
+Norlander
+Norkus
+Norgaard
+Norena
+Nored
+Nobrega
+Niziolek
+Ninnemann
+Nievas
+Nieratko
+Nieng
+Niedermeyer
+Niedermaier
+Nicolls
+Niang
+Newham
+Newcome
+Newberger
+Nevills
+Nevens
+Nevel
+Neumiller
+Netti
+Net
+Nessler
+Neria
+Nemet
+Nelon
+Nellon
+Neller
+Neisen
+Neilly
+Neifer
+Neid
+Negro
+Neering
+Neehouse
+Neef
+Needler
+Nebergall
+Nealis
+Naumoff
+Naufzinger
+Narum
+Narro
+Narramore
+Naraine
+Napps
+Nansteel
+Namisnak
+Namanny
+Nallie
+Nakhle
+Naito
+Naccari
+Nabb
+Myracle
+Myra
+Myhand
+Mwakitwile
+Muzzy
+Muscolino
+Musco
+Muscente
+Muscat
+Muscara
+Musacchia
+Musa
+Murrish
+Murfin
+Muray
+Munnelly
+Munley
+Munivez
+Mundine
+Mundahl
+Munari
+Mulling
+Mullennex
+Mullendore
+Mulkhey
+Mulinix
+Mulders
+Muhl
+Muenchow
+Muellner
+Mudget
+Mudger
+Muckenfuss
+Muchler
+Mozena
+Movius
+Mouldin
+Motola
+Mosseri
+Mossa
+Moselle
+Mory
+Morsell
+Morrish
+Morles
+Morie
+Morguson
+Moresco
+Morck
+Moppin
+Moosman
+Moons
+Montuori
+Montono
+Montogomery
+Montis
+Monterio
+Monter
+Monsalve
+Mongomery
+Mongar
+Mondello
+Moncivais
+Monard
+Monagan
+Molt
+Mollenhauer
+Moldrem
+Moldonado
+Molano
+Mokler
+Moisant
+Moilanen
+Mohrman
+Mohamad
+Moger
+Mogel
+Modine
+Modin
+Modic
+Modha
+Modena
+Mlynek
+Miya
+Mittiga
+Mittan
+Mitcheltree
+Miss
+Misfeldt
+Misener
+Mirchandani
+Miralles
+Miotke
+Miosky
+Minty
+Mintey
+Mins
+Minnie
+Mince
+Minassian
+Minar
+Mimis
+Milon
+Milloy
+Millison
+Milito
+Milfort
+Milbradt
+Mikulich
+Mikos
+Miklas
+Mihelcic
+Migliorisi
+Migliori
+Miesch
+Midura
+Miclette
+Michele
+Michela
+Micale
+Mezey
+Mews
+Mewes
+Mettert
+Mesker
+Mesich
+Mesecher
+Merthie
+Mersman
+Mersereau
+Merrithew
+Merriott
+Merring
+Merenda
+Merchen
+Mercardo
+Merati
+Mentzel
+Mentis
+Mentel
+Menotti
+Meno
+Mengle
+Mendolia
+Mellick
+Mellett
+Melichar
+Melhorn
+Melendres
+Melchiorre
+Meitzler
+Mehtani
+Mehrtens
+Megan
+Meditz
+Medeiras
+Meckes
+Me
+Mcteer
+Mctee
+Mcparland
+Mcniell
+Mcnealey
+Mcmanaway
+Mcleon
+Mclay
+Mclavrin
+Mcklveen
+Mckinzey
+Mcken
+Mckeand
+Mckale
+Mcilwraith
+Mcilroy
+Mcgreal
+Mcgougan
+Mcgettigan
+Mcgarey
+Mcfeeters
+Mcelhany
+Mcdaris
+Mccomis
+Mccomber
+Mccolm
+Mccollins
+Mccollin
+Mccollam
+Mccoach
+Mcclory
+Mcclennon
+Mccathern
+Mccarthey
+Mccarson
+Mccarrel
+Mccargar
+Mccandles
+Mccamish
+Mccally
+Mccage
+Mcbrearty
+Mcaneny
+Mcanallen
+Mcalarney
+Mcaferty
+Mazzo
+Mazy
+Mazurowski
+Mazique
+Mayoras
+Mayden
+Maxberry
+Mauller
+Matusiak
+Mattsen
+Matthey
+Matters
+Matkins
+Mathiasen
+Mathe
+Mateus
+Mate
+Matalka
+Masullo
+Massay
+Mashak
+Mascroft
+Martinex
+Martenson
+Marsiglia
+Marsella
+Marseille
+Maroudas
+Marotte
+Marner
+Marlo
+Markes
+Marina
+Maret
+Mareno
+Marean
+Marcinkiewicz
+Marchel
+Marasigan
+Manzueta
+Manzanilla
+Manternach
+Manring
+Manquero
+Manoni
+Manne
+Mankowski
+Manjarres
+Mangen
+Mangat
+Mandonado
+Mandia
+Mancias
+Manbeck
+Mamros
+Mam
+Maltez
+Mallia
+Mallar
+Malla
+Mall
+Malen
+Malaspina
+Malahan
+Malagisi
+Malachowski
+Makowsky
+Makinen
+Makepeace
+Majkowski
+Majid
+Majestic
+Majercin
+Maisey
+Mainguy
+Mailliard
+Maignan
+Mahlman
+Maha
+Magsamen
+Magpusao
+Magnano
+Magley
+Magedanz
+Magarelli
+Magaddino
+Maenner
+Madnick
+Maddrey
+Madaffari
+Macnaughton
+Macmullen
+Macksey
+Macknight
+Macki
+Macisaac
+Maciejczyk
+Maciag
+Macho
+Machenry
+Machamer
+Macguire
+Macdougal
+Macdaniel
+Maccormack
+Maccabe
+Mabbott
+Mabb
+Lynott
+Lyndon
+Lym
+Lydia
+Lycan
+Luy
+Lutwin
+Luscombe
+Lusco
+Lusardi
+Luria
+Lunetta
+Lundsford
+Lumas
+Luisi
+Luevanos
+Lueckenhoff
+Ludgate
+Ludd
+Lucherini
+Lubbs
+Lozado
+Lovie
+Lourens
+Lounsberry
+Loughrey
+Loughary
+Lotton
+Losser
+Loshbaugh
+Loser
+Loseke
+Loscalzo
+Los
+Lortz
+Loperena
+Loots
+Loosle
+Looman
+Longstaff
+Longobardi
+Longbottom
+Lomay
+Lomasney
+Lohrmann
+Lohmiller
+Logalbo
+Loetz
+Loeffel
+Lodwick
+Lodrigue
+Lockrem
+Llera
+Llarena
+Liv
+Littrel
+Littmann
+Lisser
+Lippa
+Lipner
+Linnemann
+Lingg
+Lindemuth
+Lindeen
+Limbo
+Lillig
+Likins
+Lights
+Lieurance
+Liesmann
+Liesman
+Liendo
+Lickert
+Lichliter
+Leyvas
+Leyrer
+Lewy
+Leubner
+Letters
+Lesslie
+Lesnick
+Lesmerises
+Lerno
+Lequire
+Lepera
+Lepard
+Lenske
+Leneau
+Lempka
+Lemmen
+Lemm
+Lemere
+Leinhart
+Leichner
+Leicher
+Leibman
+Lehmberg
+Leggins
+Lebeda
+Leavengood
+Leanard
+Lazaroff
+Laventure
+Lavant
+Lauster
+Laumea
+Latigo
+Lasota
+Lashure
+Lasecki
+Lascurain
+Lartigue
+Larouche
+Lappe
+Laplaunt
+Laplace
+Lanum
+Lansdell
+Lanpher
+Lanoie
+Lankard
+Laniado
+Langowski
+Langhorn
+Langfield
+Langfeldt
+Landt
+Landingham
+Landerman
+Landavazo
+Lampo
+Lampke
+Lamper
+Lamery
+Lambey
+Lamadrid
+Lallemand
+Laisure
+Laigo
+Laguer
+Lagerman
+Lageman
+Lagares
+Lacosse
+Lachappelle
+Labs
+Laborn
+Labonne
+Kyung
+Kuzia
+Kutt
+Kutil
+Kus
+Kurylo
+Kurowski
+Kuriger
+Kupcho
+Kulzer
+Kulesa
+Kules
+Kuhs
+Kuhne
+Krutz
+Krus
+Krupka
+Kronberg
+Kromka
+Kroese
+Krizek
+Krivanek
+Krishna
+Kringel
+Kreiss
+Kratofil
+Krapp
+Krakowsky
+Kracke
+Kozlow
+Koy
+Kowald
+Kover
+Kovaleski
+Kothakota
+Kosten
+Koskinen
+Kositzke
+Korff
+Korey
+Korbar
+Kor
+Kopplin
+Koplin
+Koos
+Konyn
+Konczak
+Komp
+Komo
+Kolber
+Kolash
+Kolakowski
+Kohm
+Kogen
+Koestner
+Koegler
+Kodama
+Kocik
+Kochheiser
+Kobler
+Kobara
+Knezevich
+Kneifl
+Knapchuck
+Knabb
+Klutz
+Klugman
+Klosner
+Klingel
+Klimesh
+Klice
+Kley
+Kleppe
+Klemke
+Kleinmann
+Kleinhans
+Kleinberg
+Kleffner
+Kleckley
+Klase
+Kisto
+Kissick
+Kisselburg
+Kirsten
+Kirschman
+Kirks
+Kirkner
+Kirkey
+Kirchman
+Kipling
+Kinville
+Kinnunen
+Kingdom
+Kimmey
+Kimmerle
+Kimbley
+Kilty
+Kilts
+Killmeyer
+Killilea
+Killay
+Kiest
+Kierce
+Kiepert
+Kielman
+Khalid
+Kewal
+Keszler
+Kesson
+Kesich
+Kerwood
+Kerksiek
+Kerkhoff
+Kerbo
+Keranen
+Keomuangtai
+Kenter
+Kennelley
+Keniry
+Kendzierski
+Kempner
+Kemmis
+Kemerling
+Kelsay
+Kelchner
+Kela
+Keithly
+Keipe
+Kegg
+Keer
+Keahey
+Kaywood
+Kayes
+Kawahara
+Kasuboski
+Kastendieck
+Kassin
+Kasprzyk
+Karraker
+Karnofski
+Karman
+Karger
+Karge
+Karella
+Karbowski
+Kapphahn
+Kap
+Kannel
+Kamrath
+Kaminer
+Kamansky
+Kalua
+Kaltz
+Kalpakoff
+Kalkbrenner
+Kaku
+Kaib
+Kaehler
+Kackley
+Kaber
+Justo
+Juris
+Jurich
+Jurgenson
+Jurez
+Junor
+Juniel
+Juncker
+Jugo
+Jubert
+Jowell
+Jovanovic
+Josiah
+Joosten
+Joncas
+Joma
+Johnso
+Johanns
+Jodoin
+Jockers
+Joans
+Jinwright
+Jinenez
+Jimeson
+Jerrett
+Jergens
+Jerden
+Jerdee
+Jepperson
+Jendras
+Jeanfrancois
+Jazwa
+Jaussi
+Jaster
+Jarzombek
+Jarencio
+Janocha
+Jakab
+Jadlowiec
+Jacobsma
+Jach
+Izaquirre
+Iwaoka
+Ivaska
+Iturbe
+Israelson
+Ismael
+Isles
+Isachsen
+Isaak
+Irland
+Inzerillo
+Insogna
+Ingegneri
+Ingalsbe
+Inciong
+Inagaki
+Idol
+Icenogle
+Hyon
+Hyett
+Hyers
+Huyck
+Hutti
+Hutten
+Hutnak
+Hussar
+Husky
+Hurrle
+Hurford
+Hurde
+Hupper
+Hunkin
+Hunkele
+Hunke
+Hun
+Humann
+Huhtasaari
+Hugger
+Hugel
+Huge
+Hufft
+Huegel
+Hrobsky
+Hren
+Hoyles
+Howlin
+Hovsepian
+Hovenga
+Hovatter
+Houdek
+Hotze
+Hossler
+Hossfeld
+Hosseini
+Horten
+Hort
+Horr
+Horgen
+Horen
+Hoopii
+Hoon
+Hoogland
+Hontz
+Honnold
+Homewood
+Holway
+Holtgrewe
+Holtan
+Holstrom
+Holstege
+Hollway
+Hollingshed
+Holling
+Hollenback
+Hollard
+Holberton
+Hoines
+Hogeland
+Hofstad
+Hoetger
+Hoen
+Hoaglund
+Hirota
+Hintermeister
+Hinnen
+Hinders
+Hinderer
+Hinchee
+Himelfarb
+Himber
+Hilzer
+Hilling
+Hillers
+Hillegas
+Hildinger
+Hignight
+Highman
+Hierholzer
+Heyde
+Hettich
+Hesketh
+Herzfeld
+Herzer
+Hershenson
+Hershberg
+Hernando
+Hermenegildo
+Hereth
+Hererra
+Hereda
+Herbin
+Heraty
+Herard
+Hepa
+Henschel
+Henrichsen
+Hennes
+Henneberger
+Heningburg
+Henig
+Hendron
+Hendericks
+Hemple
+Hempe
+Hemmingsen
+Hemler
+Helvie
+Helmly
+Helmbrecht
+Heling
+Helin
+Helfrey
+Helble
+Helaire
+Heizman
+Heisser
+Heiny
+Heinbaugh
+Heigh
+Heidemann
+Heidema
+Heiberger
+Hegel
+Heerdt
+Heeg
+Heefner
+Heckerman
+Heckendorf
+Heavin
+Headman
+Haynesworth
+Haylock
+Hayakawa
+Hawksley
+Hawking
+Haverstick
+Haut
+Hausen
+Hauke
+Haubold
+Hattan
+Hattabaugh
+Hasten
+Hasstedt
+Hashem
+Haselhorst
+Harrist
+Harpst
+Haroldsen
+Harmison
+Harkema
+Hark
+Harison
+Hariri
+Harcus
+Harcum
+Harcourt
+Harcharik
+Hanzel
+Hanvey
+Hantz
+Hansche
+Hansberger
+Hannig
+Hanken
+Hanhardt
+Hanf
+Hanauer
+Hamberlin
+Halward
+Halsall
+Hals
+Hallquist
+Hallmon
+Halk
+Halbach
+Halat
+Hajdas
+Hainsworth
+Haik
+Hahm
+Hagger
+Haggar
+Hader
+Hadel
+Haddick
+Hackmann
+Haasch
+Haaf
+Guzzetta
+Guzy
+Gutterman
+Gutmann
+Gutkowski
+Gustine
+Gursky
+Gurner
+Gunsolley
+Gumpert
+Gumbel
+Gulla
+Guilmain
+Guiliani
+Guier
+Guers
+Guerero
+Guerena
+Guebara
+Guadiana
+Grunder
+Grothoff
+Grosland
+Grosh
+Groos
+Grohs
+Grohmann
+Groepper
+Grodi
+Grizzaffi
+Grissinger
+Grippi
+Grinde
+Griffee
+Grether
+Greninger
+Greigo
+Gregorski
+Greger
+Grega
+Greenberger
+Graza
+Grattan
+Grasse
+Gras
+Grano
+Gramby
+Gradilla
+Govin
+Goutremout
+Goulas
+Gotay
+Gosling
+Gorey
+Goren
+Gordner
+Goossen
+Goon
+Goodwater
+Gonzaga
+Gonyo
+Gonska
+Gongalves
+Gomillion
+Gombos
+Golonka
+Gollman
+Goldtrap
+Goldammer
+Golas
+Golab
+Gola
+Gogan
+Goffman
+Goeppinger
+Godkin
+Godette
+Glore
+Glomb
+Glauner
+Glassey
+Glasner
+Gividen
+Giuffrida
+Gishal
+Giovanelli
+Ginoza
+Ginns
+Gindlesperger
+Gindhart
+Gillem
+Gilger
+Giggey
+Giebner
+Gibbson
+Giacomo
+Giacolone
+Giaccone
+Giacchino
+Ghere
+Gherardini
+Gherardi
+Gfeller
+Getts
+Gerwitz
+Gervin
+Gerstle
+Gerfin
+Geremia
+Gercak
+General
+Gener
+Gencarelli
+Gehron
+Gehrmann
+Geffers
+Geery
+Geater
+Gawlik
+Gaudino
+Garsia
+Garrahan
+Garrabrant
+Garofolo
+Garigliano
+Garfinkle
+Garelick
+Gardocki
+Garafola
+Gappa
+Gantner
+Ganther
+Gangelhoff
+Gamarra
+Galstad
+Gally
+Gallik
+Gallier
+Galimba
+Gali
+Galassi
+Gaige
+Gadsby
+Gabby
+Gabbin
+Gabak
+Fyall
+Furney
+Funez
+Fulwider
+Fulson
+Fukunaga
+Fujikawa
+Fugere
+Fuertes
+Fuda
+Fryson
+Frump
+Frothingham
+Froning
+Froncillo
+Frohling
+Froberg
+Froats
+Fritchman
+Frische
+Friedrichsen
+Friedmann
+Fridge
+Friddell
+Frid
+Fresch
+Frentzel
+Freno
+Frelow
+Freimuth
+Freidel
+Freehan
+Freeby
+Freeburn
+Fredieu
+Frederiksen
+Fredeen
+Frazell
+Frayser
+Fratzke
+Frattini
+Franze
+Franich
+Francescon
+Francesco
+Frames
+Framer
+Fraiser
+Fragman
+Frack
+Foxe
+Fowlston
+Fosberg
+Fortna
+Fornataro
+Forden
+Foots
+Foody
+Fogt
+Foglia
+Fogerty
+Fogelson
+Flygare
+Flowe
+Florentine
+Flinner
+Flem
+Flatten
+Flath
+Flater
+Flahaven
+Flad
+Fjeld
+Fitanides
+Fistler
+Fishbaugh
+Firsching
+Fireman
+Finzel
+Finical
+Fingar
+Filosa
+Filicetti
+Filby
+Fierst
+Fierra
+Ficklen
+Ficher
+Fersner
+Ferrufino
+Ferrucci
+Fero
+Ferns
+Ferlenda
+Ferko
+Fergerstrom
+Ferge
+Fenty
+Fent
+Fennimore
+Fendt
+Femat
+Felux
+Felman
+Feldhaus
+Feisthamel
+Feijoo
+Feiertag
+Fehrman
+Fehl
+Feezell
+Feeny
+Feeback
+Fedigan
+Fedder
+Fechner
+Feary
+Fayson
+Faylor
+Fauteux
+Faustini
+Faure
+Fauci
+Fauber
+Fattig
+Farruggio
+Farrens
+Fare
+Faraci
+Fantini
+Fantin
+Fanno
+Fannings
+Faniel
+Fallaw
+Falker
+Falkenhagen
+Fajen
+Fahrner
+Fabel
+Fabacher
+Eytcheson
+Eyster
+Exford
+Exel
+Exe
+Evetts
+Evenstad
+Evanko
+Euresti
+Euber
+Etcitty
+Estler
+Esther
+Essner
+Essinger
+Esplain
+Espenshade
+Espanol
+Espaillat
+Escribano
+Escorcia
+Errington
+Errett
+Errera
+Erlanger
+Erenrich
+Erekson
+Erber
+Entinger
+Ensworth
+Ensell
+Enno
+Ennen
+Englin
+Engblom
+Engberson
+Encinias
+Enama
+Emel
+Elzie
+Elsbree
+Elmo
+Elman
+Elm
+Ellebracht
+Elkan
+Elfstrom
+Elerson
+Eleazer
+Eleam
+Eldrige
+Elcock
+Einspahr
+Eike
+Eidschun
+Eid
+Eickman
+Eichele
+Eiche
+Ehlke
+Eguchi
+Eggink
+Edouard
+Edgehill
+Eckes
+Eblin
+Ebberts
+Eavenson
+Earvin
+Eardley
+Eagon
+Eader
+Dzubak
+Dylla
+Dyckman
+Dwire
+Dutrow
+Dutile
+Dusza
+Dustman
+Dusing
+Duryee
+Durupan
+Durtschi
+Durtsche
+Durell
+Dunny
+Dunnegan
+Dunken
+Dun
+Dumm
+Dulak
+Duker
+Dukelow
+Dufort
+Dufilho
+Duffee
+Duett
+Dueck
+Dudzinski
+Dudasik
+Duckwall
+Duchemin
+Dubrow
+Dubis
+Dubicki
+Duba
+Drust
+Druckman
+Drinnen
+Drewett
+Drewel
+Dreitzler
+Dreckman
+Drappo
+Draffen
+Drabant
+Doyen
+Dowding
+Doub
+Dorson
+Dorschner
+Dorrington
+Dorney
+Dormaier
+Dorff
+Dorcy
+Donges
+Donelly
+Donel
+Domangue
+Dols
+Dollahite
+Dolese
+Doldo
+Doiley
+Dohrman
+Dohn
+Doheny
+Doceti
+Dobry
+Dobrinski
+Dobey
+Divincenzo
+Dischinger
+Dirusso
+Dirocco
+Dipiano
+Diop
+Dinitto
+Dinehart
+Dimsdale
+Diminich
+Dimalanta
+Dillavou
+Dilello
+Difusco
+Diffey
+Diffenderfer
+Diffee
+Difelice
+Difabio
+Dietzman
+Dieteman
+Diepenbrock
+Dieckmann
+Dicey
+Dicampli
+Dibari
+Diazdeleon
+Diallo
+Dewitz
+Dewiel
+Devoll
+Devol
+Devincent
+Devier
+Devendorf
+Devalk
+Detten
+Detraglia
+Dethomas
+Deter
+Detemple
+Desler
+Desharnais
+Desanty
+Derocco
+Dermer
+Derks
+Derito
+Derick
+Derhammer
+Deraney
+Dequattro
+Depass
+Depadua
+Deon
+Denzel
+Denyes
+Denyer
+Dentino
+Denlinger
+Deneal
+Demory
+Demopoulos
+Demontigny
+Demonte
+Demeza
+Delsol
+Delrosso
+Delpit
+Delpapa
+Delouise
+Delone
+Delo
+Delmundo
+Delmore
+Delmar
+Dellapaolera
+Delfin
+Delfierro
+Deleonardis
+Delenick
+Delcarlo
+Delcampo
+Delcamp
+Delawyer
+Delaware
+Delaroca
+Delaluz
+Delahunt
+Delaguardia
+Dekeyser
+Dekay
+Dejaeger
+Dejackome
+Dehay
+Dehass
+Degraffenried
+Degenhart
+Degan
+Deever
+Deedrick
+Deckelbaum
+Dechico
+Decent
+Dececco
+Decasas
+Debrock
+Debona
+Debeaumont
+Debarros
+Debaca
+Dearmore
+Deangelus
+Dealmeida
+Dawood
+Davney
+Daudt
+Datri
+Dasgupta
+Darring
+Darracott
+Darius
+Darcus
+Daoud
+Dansbury
+Dannels
+Danish
+Danielski
+Danehy
+Dancey
+Damour
+Dambra
+Daman
+Dalcour
+Daisey
+Dahlheimer
+Dagon
+Dadisman
+Dacunto
+Dacamara
+Dabe
+Cyrulik
+Cyphert
+Cwik
+Cussen
+Curles
+Curit
+Curby
+Curbo
+Cunas
+Cunard
+Cunanan
+Cumpton
+Culcasi
+Cui
+Cucinotta
+Cucco
+Csubak
+Cruthird
+Crumwell
+Crummitt
+Crumedy
+Crouthamel
+Cronce
+Cromack
+Cristina
+Crisafi
+Crimin
+Cresto
+Crescenzo
+Cremonese
+Creedon
+Credit
+Crankshaw
+Cozzens
+Cove
+Coval
+Courtwright
+Courcelle
+Coupland
+Counihan
+Coullard
+Cotrell
+Cosgrave
+Cornfield
+Cornelio
+Corish
+Cordoua
+Corbit
+Coppersmith
+Coonfield
+Cools
+Conville
+Contrell
+Contento
+Conser
+Conrod
+Connole
+Congrove
+Conery
+Condray
+Colver
+Coltman
+Colflesh
+Colcord
+Colavito
+Colar
+Coile
+Coggan
+Coenen
+Codling
+Coda
+Cockroft
+Cockrel
+Cockerill
+Cocca
+Coberley
+Coaster
+Clouden
+Clos
+Clive
+Clish
+Clint
+Clinkscale
+Clester
+Clammer
+City
+Cittadino
+Citrano
+Ciresi
+Cillis
+Ciccarelli
+Ciborowski
+Ciarlo
+Ciardullo
+Chritton
+Chopp
+Choo
+Chirco
+Chilcoat
+Chevarie
+Cheslak
+Chernak
+Chay
+Chatterjee
+Chatten
+Chatagnier
+Chastin
+Chappuis
+Channing
+Channey
+Champlain
+Chalupsky
+Chalfin
+Chaffer
+Chadek
+Chadderton
+Cestone
+Cestero
+Cestari
+Cerros
+Cermeno
+Centola
+Cedrone
+Cayouette
+Cavan
+Cavaliero
+Casuse
+Castricone
+Castoreno
+Casten
+Castanada
+Castagnola
+Casstevens
+Cassio
+Cassi
+Cassanova
+Caspari
+Casher
+Cashatt
+Casco
+Casassa
+Casad
+Carville
+Carvel
+Cartland
+Cartegena
+Carsey
+Carsen
+Carrino
+Carrilo
+Carpinteyro
+Carmley
+Carlston
+Carlsson
+Carie
+Cariddi
+Caricofe
+Carel
+Cardy
+Carducci
+Carby
+Carangelo
+Capriotti
+Capria
+Caprario
+Capelo
+Canul
+Cantua
+Cantlow
+Canny
+Cangialosi
+Canepa
+Candland
+Campolo
+Campi
+Camors
+Camino
+Camfield
+Camelo
+Camarero
+Camaeho
+Calvano
+Callum
+Calliste
+Caldarella
+Calcutt
+Calcano
+Caissie
+Cager
+Caccamo
+Cabotage
+Cabble
+Byman
+Buzby
+Butkowski
+Bussler
+Busico
+Bushy
+Bushovisky
+Busbin
+Busard
+Busalacchi
+Burtman
+Burrous
+Burridge
+Burrer
+Burno
+Burin
+Burgette
+Burdock
+Burdier
+Burckhard
+Bunten
+Bungay
+Bundage
+Bumby
+Bultema
+Bulinski
+Bulan
+Bukhari
+Buganski
+Buerkle
+Buen
+Buehl
+Bue
+Budzynski
+Buckham
+Bub
+Bryk
+Brydon
+Bruyere
+Brunsvold
+Brunnett
+Brunker
+Brunfield
+Brumble
+Brue
+Brozina
+Brossman
+Brosey
+Brookens
+Broersma
+Brodrick
+Brockmeier
+Brockhouse
+Brisky
+Brinkly
+Brine
+Brincefield
+Brighenti
+Brigante
+Brieno
+Briede
+Bridenbaugh
+Bridegroom
+Brickett
+Bria
+Breske
+Brener
+Brenchley
+Breitkreutz
+Breitbart
+Breister
+Breining
+Breighner
+Breidel
+Brehon
+Breheny
+Breard
+Brean
+Breakell
+Breach
+Brazill
+Braymiller
+Braum
+Brau
+Brashaw
+Bransom
+Brandolino
+Brancato
+Branagan
+Braff
+Brading
+Bracker
+Brackenbury
+Bracher
+Braasch
+Boylen
+Boyda
+Boyanton
+Bowlus
+Bowditch
+Boutot
+Bouthillette
+Boursiquot
+Bourjolly
+Bouret
+Bouquet
+Boulerice
+Bouer
+Bouchillon
+Bouchie
+Bottin
+Boteilho
+Bosko
+Bosack
+Borys
+Bors
+Borla
+Borjon
+Borghi
+Borah
+Booty
+Booten
+Boore
+Bonuz
+Bonne
+Bongers
+Boneta
+Bonawitz
+Bonanni
+Bomer
+Bollen
+Bollard
+Bolla
+Bolio
+Boisseau
+Boies
+Boiani
+Bohorquez
+Boghossian
+Boespflug
+Boeser
+Boehl
+Boegel
+Bodrick
+Bodkins
+Bodenstein
+Bodell
+Bockover
+Bocci
+Bobbs
+Boals
+Boahn
+Boadway
+Bluma
+Bluett
+Bloor
+Blomker
+Blevens
+Blethen
+Bleecker
+Blayney
+Blaske
+Blasetti
+Blancas
+Blackner
+Blackie
+Bjorkquist
+Bjerk
+Bizub
+Bisono
+Bisges
+Bisaillon
+Birr
+Birnie
+Bires
+Birdtail
+Birdine
+Bina
+Billock
+Billinger
+Billig
+Billet
+Bigwood
+Bigalk
+Bielicki
+Biddick
+Biccum
+Biafore
+Bhagat
+Beza
+Beyah
+Bex
+Bevier
+Bevell
+Beute
+Betzer
+Betthauser
+Bethay
+Bethard
+Beshaw
+Bertholf
+Bertels
+Berridge
+Bernot
+Bernath
+Bernabei
+Berkson
+Berkovitz
+Berkich
+Bergsten
+Berget
+Berezny
+Berdin
+Beougher
+Benthin
+Benhaim
+Benenati
+Benejan
+Bemiss
+Beloate
+Bellucci
+Bells
+Bellotti
+Belling
+Bellido
+Bellaire
+Bellafiore
+Bekins
+Bekele
+Beish
+Behnken
+Beerly
+Beddo
+Becket
+Becke
+Bebeau
+Beauchaine
+Beaucage
+Beadling
+Beacher
+Bazar
+Baysmore
+Bayers
+Baun
+Baulch
+Baucher
+Batto
+Baton
+Bathe
+Basora
+Baruffi
+Bartimus
+Bartholemew
+Barrickman
+Barribeau
+Barreda
+Barrack
+Baroody
+Barness
+Barn
+Barmer
+Barillari
+Barias
+Barginear
+Barg
+Barde
+Barbone
+Barbato
+Barbarin
+Baoloy
+Bansal
+Bangle
+Banducci
+Bandel
+Bambeck
+Balter
+Ballif
+Baller
+Balladares
+Balkus
+Baldy
+Baldivia
+Balcerzak
+Balazs
+Baksh
+Bakr
+Bakemeier
+Baisey
+Bainer
+Bailly
+Bagge
+Badua
+Badini
+Bachtell
+Bachrodt
+Bachorski
+Bacak
+Babula
+Bable
+Babjeck
+Babecki
+Azbell
+Ayudan
+Awai
+Avita
+Avino
+Avellar
+Auzat
+Autman
+Autio
+Autery
+Ausman
+Ausland
+Aulabaugh
+Augle
+Aughenbaugh
+Augeri
+Audi
+Attleson
+Attig
+Attal
+Ator
+Asselmeier
+Askland
+Asiello
+Asch
+Arya
+Artola
+Arslanian
+Arron
+Arrezola
+Arnesen
+Arnau
+Armster
+Armintrout
+Armento
+Armato
+Arkenberg
+Ariaza
+Arguin
+Arenson
+Areias
+Archut
+Archibold
+Arave
+Arand
+Appelman
+Appello
+Antonson
+Antoniewicz
+Antill
+Antigua
+Annino
+Anness
+Anneler
+Angustia
+Angry
+Angiolillo
+Angelico
+Andreula
+Andreen
+Andreassi
+Andeson
+Ander
+Anda
+Anania
+Anadio
+Amicone
+Amenta
+Alzaga
+Alwardt
+Aluarado
+Altreche
+Altic
+Alsobrooks
+Alpern
+Almodova
+Almas
+Alltop
+Alliston
+Allio
+Alipio
+Alicandro
+Alibozek
+Alguire
+Alff
+Alcalde
+Alborn
+Albery
+Alberry
+Albany
+Albani
+Albanez
+Alavi
+Akkerman
+Ahlheim
+Agresti
+Agnelli
+Agilar
+Agib
+Aggas
+Afton
+Afonso
+Adil
+Adi
+Adank
+Adamsky
+Acri
+Accurso
+Abruzzese
+Abrew
+Abeln
+Abdullai
+Abdulkarim
+Abdelrahman
+Abbenante
+Abatiell
+Abaloz
+Zyskowski
+Zwiefel
+Zurmiller
+Zupancic
+Zuno
+Zumsteg
+Zumbrennen
+Zumaya
+Zullinger
+Zuleger
+Zozaya
+Zourkos
+Zorrilla
+Zorko
+Zolocsik
+Zittel
+Ziobro
+Zimmerly
+Zimmerli
+Zillmer
+Zigmond
+Zierer
+Zieber
+Zide
+Zevenbergen
+Zephier
+Zemel
+Zelazo
+Zeitlin
+Zeiser
+Zehring
+Zeger
+Zedian
+Zearfoss
+Zbranek
+Zaya
+Zatarain
+Zasso
+Zarn
+Zarilla
+Zari
+Zapp
+Zapf
+Zanghi
+Zange
+Zamacona
+Zalesky
+Zalazar
+Zaki
+Zafar
+Zade
+Yusko
+Yurman
+Yurkovich
+Yuhasz
+Younge
+Yiu
+Yeasted
+Yarrito
+Yark
+Yarboro
+Yannuzzi
+Yankovich
+Yanagawa
+Yago
+Yaffe
+Wyndham
+Wyms
+Wyand
+Wuensch
+Wryals
+Wrubel
+Worosz
+Woolstenhulme
+Wolpe
+Wolner
+Wolgamot
+Wolfman
+Wojtaszek
+Woeppel
+Woehr
+Wodarski
+Wizwer
+Wittkop
+Wisseman
+Wisor
+Wishum
+Wischmann
+Wisch
+Wirkkala
+Wion
+Wintjen
+Wintermute
+Wintermantel
+Winks
+Winkey
+Winham
+Windschitl
+Willow
+Willitzer
+Willier
+Willets
+Willenbrink
+Willen
+Willaimson
+Wilfahrt
+Wilenkin
+Wilen
+Wildeboer
+Wilchek
+Wigren
+Wignall
+Wiggington
+Wierson
+Wiegman
+Wiegel
+Widmayer
+Wider
+Widder
+Wickey
+Wickers
+Wical
+Whiton
+Whitenton
+Whiteleather
+Whiston
+Whirley
+Whetham
+Wheatly
+Wetenkamp
+Westenberger
+Westenbarger
+Westall
+Werblow
+Wengel
+Welson
+Welschmeyer
+Wellmann
+Wellbrock
+Wela
+Wekenborg
+Weiter
+Weisenstein
+Wehmann
+Weeda
+Wede
+Webley
+Waver
+Wauford
+Waterworth
+Watchorn
+Wassinger
+Wassell
+Wasp
+Wasiuta
+Warnix
+Warning
+Warnes
+Warmoth
+Warling
+Warila
+Warga
+Warburg
+Wanzer
+Want
+Waner
+Wanek
+Walwyn
+Walle
+Walkner
+Walin
+Waletzko
+Waler
+Walenta
+Wainer
+Wailes
+Wahr
+Waddel
+Wactor
+Wachtler
+Wachsman
+Wachowski
+Vulgamore
+Vukelich
+Vote
+Vost
+Voskamp
+Vorwerk
+Vongphakdy
+Volpi
+Volle
+Volino
+Voeks
+Vodopich
+Vittone
+Virdin
+Virag
+Vinroe
+Vinegar
+Vindiola
+Vilmont
+Villerreal
+Villaneva
+Villalobas
+Villada
+Vilhauer
+Vilchis
+Vilches
+Viggiani
+Vig
+Vieux
+Viets
+Vient
+Vielle
+Viejo
+Vidovich
+Vichi
+Veys
+Veverka
+Verser
+Veronesi
+Vernoy
+Vermont
+Verhines
+Verheyen
+Veren
+Vereb
+Verano
+Venuto
+Ventry
+Ventrone
+Veltz
+Velo
+Velazguez
+Veeser
+Vassey
+Vasque
+Varin
+Varaza
+Varady
+Vaquez
+Vaquerano
+Vansteenwyk
+Vanschoick
+Vanroekel
+Vannorden
+Vanlent
+Vangrouw
+Vangelder
+Vanes
+Vanelli
+Vanderkar
+Vanderbeek
+Vandenburgh
+Vandekieft
+Vandekamp
+Vancura
+Vancooten
+Vanconey
+Vancampen
+Vanaria
+Valvano
+Vallette
+Vallero
+Valiton
+Valin
+Valeri
+Valek
+Valdovino
+Valdivieso
+Vakas
+Vagas
+Vadala
+Vaccarella
+Vacanti
+Urrabazo
+Urguhart
+Urda
+Urbino
+Urbas
+Upmeyer
+Umphlett
+Ulerio
+Uitz
+Uchimura
+Uccello
+Tysdal
+Ty
+Tweedle
+Turrubiates
+Turrubiartes
+Turri
+Turnham
+Turko
+Turben
+Tupin
+Tumulty
+Tuffey
+Tuckey
+Tuckett
+Tucholski
+Tubolino
+Tubergen
+Tsuboi
+Tschumperlin
+Tschoepe
+Trynowski
+Tryba
+Truslow
+Truog
+Trumball
+Trudelle
+Trojillo
+Trnka
+Trizarry
+Trigueiro
+Trigleth
+Tricomi
+Tresselt
+Trentacoste
+Trendell
+Trenary
+Treml
+Treleven
+Treherne
+Treasure
+Trayer
+Travino
+Traugott
+Trappey
+Tranbarger
+Tramontano
+Tramell
+Trainum
+Traino
+Traill
+Trabucco
+Townsell
+Tourtillott
+Touar
+Toscani
+Torrella
+Torguson
+Torda
+Top
+Toomes
+Tonner
+Tommasino
+Tomaro
+Tolve
+Tolefree
+Toguchi
+Tofflemire
+Tofanelli
+Tody
+Toce
+Tobacco
+Toan
+Toalson
+Tkacik
+Tirone
+Tipple
+Tippery
+Tinson
+Tinnell
+Timper
+Timmers
+Times
+Timblin
+Tilotta
+Tillberg
+Tijernia
+Tigges
+Tigar
+Tielking
+Thyng
+Thonen
+Thomley
+Thombs
+Thimmesch
+Thier
+Thevenin
+Theodorov
+Theodoropoulo
+Tharnish
+Tharaldson
+Thackaberry
+Tewari
+Tetu
+Tetter
+Tersigni
+Tepezano
+Tennon
+Tennent
+Teichman
+Teehan
+Tayloe
+Taus
+Tatis
+Tata
+Tat
+Tashima
+Tarufelli
+Tarlow
+Tarkowski
+Tarka
+Targett
+Taran
+Tarabokija
+Tappen
+Tanzer
+Tanous
+Tanigawa
+Taneja
+Tammo
+Tallerico
+Tallada
+Talk
+Talhelm
+Takehara
+Takata
+Tagliavia
+Taffer
+Tadman
+Tacdol
+Tacconi
+Tables
+Szewczak
+Szeredy
+Szanto
+Sympson
+Symmes
+Syers
+Sydney
+Syas
+Swinny
+Swierk
+Swendsen
+Sweigard
+Sweezey
+Sweesy
+Sween
+Sweely
+Sweed
+Sweazy
+Swauger
+Swansbrough
+Swango
+Swanda
+Swamp
+Swallows
+Swaggerty
+Svatek
+Survant
+Surowka
+Surina
+Suozzi
+Sunstrom
+Sunford
+Sundseth
+Sundahl
+Summerill
+Sumida
+Sumbler
+Suma
+Sulyma
+Sulla
+Sulieman
+Suit
+Sugiyama
+Suell
+Sudo
+Suddreth
+Sucher
+Sturn
+Sturkey
+Studzinski
+Studler
+Stuckmeyer
+Stryjewski
+Stroy
+Strotman
+Strollo
+Stroik
+Stroede
+Streeby
+Stredny
+Strazi
+Stray
+Strawderman
+Straiton
+Stower
+Stoudmire
+Stormont
+Stopka
+Stoneback
+Stoldt
+Stolarz
+Stolarski
+Stockmaster
+Stobb
+Stivason
+Stirk
+Stipp
+Stipes
+Stingel
+Stike
+Stiebel
+Stidd
+Steurer
+Sterley
+Sterle
+Stepro
+Stepovich
+Stephson
+Stenseth
+Stenerson
+Stello
+Steinbrook
+Steidley
+Stehlin
+Stegmaier
+Stefanow
+Steese
+Steenhuis
+Stavely
+Stave
+Stautz
+Staunton
+Stater
+Stas
+Startup
+Startt
+Startin
+Starratt
+Stargell
+Starcevich
+Stank
+Stanis
+Standing
+Stancliff
+Stanchfield
+Stanbrough
+Stakes
+Stahmer
+Staheli
+Staebell
+Stadtlander
+Stadheim
+Sroufe
+Sroczynski
+Srnsky
+Sreaves
+Srader
+Squeo
+Spuler
+Sproat
+Springmeyer
+Sprengeler
+Sport
+Spolar
+Spivack
+Spinale
+Spiegler
+Spickerman
+Spessard
+Spenner
+Speich
+Spaziano
+Sparaco
+Spalter
+Sowells
+Sovich
+Southmayd
+Southgate
+Sotto
+Sotomayer
+Sosaya
+Sorvillo
+Sorrel
+Soos
+Songco
+Somerset
+Somero
+Soll
+Soldan
+Solarzano
+Solana
+Sokal
+Soibelman
+Soesbe
+Sobotta
+Sobina
+Sobeck
+Soard
+Snorton
+Snopek
+Snoozy
+Snethen
+Smithhisler
+Smee
+Smaniotto
+Slusarski
+Slowe
+Slotnick
+Sleva
+Sleighter
+Slappey
+Skyers
+Skutt
+Skorcz
+Skoczylas
+Skillicorn
+Skiffington
+Skibicki
+Skerl
+Skehan
+Skalla
+Siwinski
+Sivley
+Sittloh
+Sitterly
+Sith
+Sit
+Sise
+Siroky
+Sirles
+Sirin
+Sirignano
+Siren
+Sinsabaugh
+Sinks
+Sinisi
+Sinibaldi
+Singson
+Sindlinger
+Simpkin
+Siminski
+Simcoe
+Siford
+Siegert
+Sidor
+Sidhom
+Siddique
+Siddell
+Sicotte
+Sichting
+Sicari
+Sic
+Siano
+Shufflebarger
+Shramek
+Shortnacy
+Sholler
+Sholette
+Sholders
+Shogren
+Shoenberger
+Shoemate
+Shoat
+Shinoda
+Shines
+Shimshak
+Shigley
+Sheward
+Shetrone
+Shetlar
+Sherretts
+Sherod
+Shenkle
+Shely
+Sheltra
+Shelpman
+Shellabarger
+Shelite
+Sheldrick
+Shelburn
+Sheinbein
+Shebby
+Shawley
+Shatrau
+Shartle
+Sharifi
+Shanker
+Shami
+Shamel
+Shamburg
+Shamas
+Shallow
+Shaffstall
+Shadowens
+Shackleton
+Shaak
+Seykora
+Seyfert
+Sevillano
+Sevcik
+Seubert
+Seu
+Setter
+Sesler
+Servatius
+Serrant
+Serramo
+Serl
+Serini
+Serenil
+Serapion
+Sept
+Sensibaugh
+Sens
+Senich
+Sengbusch
+Sendra
+Senate
+Semrau
+Semrad
+Sempertegui
+Semons
+Semke
+Selma
+Sellinger
+Seliga
+Sekel
+Seilheimer
+Seigfried
+Seesholtz
+Seefeld
+Seecharran
+Sedrakyan
+Seavy
+Search
+Seamster
+Seabold
+Scyoc
+Sculley
+Scullawl
+Scrogham
+Scow
+Scopa
+Scontras
+Sciulli
+Sciola
+Scifres
+Schweyen
+Schwering
+Schwerdtfeger
+Schweim
+Schweikert
+Schweder
+Schwebel
+Schwartzwalde
+Schusterman
+Schuhmann
+Schuerman
+Schuchman
+Schrotenboer
+Schreurs
+Schoppert
+Schopper
+Schools
+Schoneman
+Scholfield
+Schoeppner
+Schoenleber
+Schoeman
+Schoel
+Schnurbusch
+Schnepel
+Schnader
+Schlarb
+Schlappi
+Schlangen
+Schlaht
+Schiraldi
+Schinkel
+Schimizzi
+Schifo
+Schiesher
+Scheyer
+Schettler
+Scheppke
+Schepper
+Scheinost
+Scheidel
+Scheets
+Schatzman
+Scharwath
+Scharp
+Schaarschmidt
+Schaack
+Scarnato
+Scarnati
+Scaringi
+Scarcia
+Scarano
+Sberna
+Sawina
+Sawer
+Sawaya
+Sawatzky
+Savcedo
+Sauser
+Saumier
+Sauchez
+Sauceman
+Sathre
+Satawa
+Sasala
+Sartoris
+Sare
+Sarchet
+Saracco
+Santulli
+Santory
+Santorelli
+Santopietro
+Sansing
+Sanseverino
+Saniatan
+Sangiacomo
+Sanges
+Sanfratello
+Sanflippo
+Sandona
+Sandelin
+Sandate
+Samona
+Sammis
+Sambor
+Samano
+Salvitti
+Salvietti
+Salvi
+Salum
+Salsa
+Salonek
+Salm
+Salles
+Sall
+Salera
+Salemo
+Salee
+Salak
+Sakihara
+Sakasegawa
+Sakaguchi
+Sagastegui
+Saeturn
+Sadan
+Sacayanan
+Saborio
+Sabeiha
+Sabedra
+Sabagh
+Rzepecki
+Rzasa
+Ryser
+Ryner
+Rydman
+Rycroft
+Rybij
+Ruyes
+Ruttan
+Russon
+Rushe
+Rusert
+Rusell
+Runnells
+Rundstrom
+Rumschlag
+Rullman
+Ruka
+Ruiloba
+Ruh
+Ruggs
+Ruffer
+Ruest
+Rueluas
+Rueger
+Ruediger
+Rubinoff
+Rubendall
+Rozmus
+Roxburgh
+Rowls
+Rousch
+Rothove
+Rotelli
+Roszel
+Roske
+Roskam
+Rosensteel
+Rosendo
+Roome
+Rombough
+Romash
+Romanson
+Romanello
+Romance
+Rolison
+Rogol
+Rogas
+Roese
+Roehrs
+Roegner
+Roeger
+Rodrguez
+Rodeman
+Rodebaugh
+Rockenbaugh
+Rocconi
+Robleto
+Robateau
+Roarty
+Roaf
+Rivenberg
+Rivara
+Rivali
+Risse
+Risby
+Ripperger
+Riopelle
+Ringrose
+Rinebarger
+Rile
+Riggen
+Rigano
+Riff
+Rifenbark
+Rieper
+Rieffenberger
+Riedmayer
+Ridolfi
+Ridderhoff
+Rickon
+Rickers
+Rickels
+Richoux
+Richens
+Ribao
+Rhodarmer
+Rheingans
+Reznik
+Reveron
+Reus
+Reph
+Renko
+Remme
+Remlinger
+Remke
+Remily
+Reitano
+Reissig
+Reisher
+Reinitz
+Reinholtz
+Reines
+Reigstad
+Reigh
+Reichelderfer
+Rehnert
+Rehagen
+Redline
+Rediger
+Redhouse
+Redepenning
+Recla
+Rechkemmer
+Reando
+Razavi
+Rayson
+Rayna
+Rax
+Raveling
+Rauser
+Rauschenberg
+Raupach
+Raum
+Rauen
+Ratulowski
+Ratterree
+Ratering
+Rapin
+Rannels
+Rane
+Randhawa
+Ramus
+Ramsfield
+Rams
+Ramroop
+Ramano
+Raj
+Raina
+Raikes
+Ragonese
+Rafaniello
+Raetz
+Raether
+Raeside
+Radwan
+Radman
+Rademaker
+Radar
+Racki
+Rachlin
+Rabena
+Rabassa
+Rabadan
+Raad
+Quoss
+Quizon
+Quito
+Quintela
+Quimet
+Quilty
+Quilimaco
+Quidley
+Quezaire
+Quave
+Quarto
+Quaranto
+Quandel
+Qiu
+Qazi
+Pyrdum
+Pyon
+Pyeatt
+Puzinski
+Putnal
+Punter
+Pumphery
+Pumper
+Pump
+Pummell
+Pumarejo
+Pulvermacher
+Pultz
+Pully
+Pullens
+Pulkrabek
+Pulk
+Pudlinski
+Puccetti
+Przygocki
+Przybyszewski
+Prusha
+Prudente
+Prucnal
+Prottsman
+Prosch
+Prodoehl
+Procell
+Prinzivalli
+Primes
+Prey
+Presnar
+Presho
+Prentis
+Preisler
+Preisel
+Pratka
+Pratcher
+Prass
+Pozzuoli
+Powanda
+Poundstone
+Potters
+Potra
+Potestio
+Potempa
+Postlethwait
+Posas
+Portrum
+Portland
+Portilla
+Portie
+Popovitch
+Popken
+Ponzio
+Pontremoli
+Pontarelli
+Pombo
+Pomainville
+Polycarpe
+Pollart
+Politowski
+Politano
+Poliquin
+Polczynski
+Pokoj
+Poitevint
+Poissonnier
+Poeppel
+Poellot
+Poehlman
+Poehlein
+Podratz
+Pociask
+Plocher
+Pline
+Plessinger
+Plautz
+Platten
+Plass
+Plageman
+Placko
+Pizzola
+Pizzella
+Pittsenbarger
+Pittner
+Pitstick
+Pitsch
+Pitney
+Pitaniello
+Pistoresi
+Pirc
+Pinski
+Pinera
+Pincock
+Pinckley
+Pincince
+Piliero
+Pilat
+Pigue
+Pietschman
+Pierpoint
+Pierini
+Picon
+Picking
+Picardi
+Phlegm
+Phippin
+Phetteplace
+Pharel
+Pfundt
+Pfluger
+Pfeuffer
+Pfefferle
+Pezzulo
+Pezzano
+Peveler
+Pettersson
+Petsch
+Petrusky
+Petruska
+Petrulis
+Petrossian
+Petroske
+Petrini
+Petitte
+Petito
+Petela
+Petaccio
+Pesto
+Pestka
+Pesta
+Pessoa
+Perun
+Perrow
+Perricone
+Peros
+Perney
+Perlin
+Perigo
+Perella
+Percle
+Pepple
+Penz
+Penttila
+Pensiero
+Penigar
+Penez
+Pendrak
+Penas
+Pellowski
+Pellow
+Pellin
+Pelissier
+Pelini
+Pekrul
+Peevey
+Pedraja
+Pecher
+Peasel
+Payment
+Pavolini
+Paviolitis
+Paulsell
+Paulina
+Paule
+Patrum
+Patrone
+Patrie
+Patras
+Patera
+Patek
+Patane
+Pastrano
+Pastora
+Passow
+Passley
+Passaretti
+Passantino
+Paske
+Partible
+Parsa
+Parnes
+Parliman
+Parlato
+Paravati
+Paradowski
+Papaleo
+Papagni
+Paoletta
+Panzarino
+Pannunzio
+Panis
+Pandit
+Paluzzi
+Palomin
+Palomaki
+Pallanes
+Palla
+Pall
+Palino
+Palfreyman
+Palazzi
+Palanza
+Palagi
+Painton
+Pain
+Pahulu
+Paganico
+Paeth
+Padlo
+Padillia
+Paddy
+Paddick
+Paciolla
+Pacholski
+Paap
+Paa
+Owolabi
+Overshown
+Overocker
+Overgaard
+Ouchi
+Ottoson
+Ostrye
+Osterland
+Osland
+Oslan
+Osick
+Osen
+Osdoba
+Osberg
+Orzel
+Ortmeier
+Orren
+Ormerod
+Orio
+Orgeron
+Orengo
+Orbaker
+Opiela
+Opdahl
+Onks
+Oltrogge
+Olnick
+Olivarres
+Olide
+Oleksy
+Olaya
+Okray
+Okonek
+Okinaka
+Ojima
+Ojala
+Oinonen
+Ohotto
+Ohan
+Ogwin
+Ogborn
+Oflaherty
+Offill
+Oetken
+Oertle
+Oehlert
+Odems
+Oconnel
+Ocha
+Ocarroll
+Oby
+Oblak
+Oberst
+Obermann
+Obas
+Oachs
+Nydegger
+Nybo
+Nuuanu
+Nutile
+Nuse
+Nuriddin
+Nungesser
+Nuber
+Noy
+Novinger
+Nouri
+Northan
+Norseworthy
+Norrod
+Normington
+Nori
+Norenberg
+Nordine
+Nop
+Noori
+Noblet
+Nives
+Nist
+Niskala
+Nilan
+Nikolai
+Nigl
+Nightengale
+Nichole
+Ni
+Nhek
+Ngvyen
+Newville
+Newsam
+Newnham
+Newmeyer
+Newlan
+Newbert
+Neuschwander
+Neusch
+Neun
+Nethken
+Nethercutt
+Nesser
+Neske
+Neman
+Nelton
+Nelles
+Nekola
+Neiling
+Neeser
+Neelly
+Nedved
+Neang
+Navejar
+Naveja
+Nauarro
+Natho
+Nathe
+Natcher
+Naser
+Nasby
+Narlock
+Nanton
+Naillon
+Naill
+Naguin
+Nagele
+Naftzger
+Naegle
+Naegele
+Naef
+Nacke
+Nabritt
+Mynhier
+Myart
+Muzquiz
+Mutty
+Musolino
+Mushero
+Murtaugh
+Murie
+Muresan
+Murdough
+Mura
+Munuz
+Munstermann
+Munsen
+Munselle
+Munise
+Mungle
+Munerlyn
+Muncher
+Mulrooney
+Mullee
+Mulaney
+Mulanax
+Muhlhauser
+Muhlestein
+Mugleston
+Mugg
+Mugford
+Muckel
+Mucerino
+Mt
+Mrotek
+Mrnak
+Mozdzierz
+Moyler
+Moury
+Moulin
+Moulding
+Moul
+Mottai
+Mostyn
+Mosimann
+Mosholder
+Mosburg
+Morrisseau
+Moron
+Morice
+Morgante
+Moreta
+Morcos
+Morasco
+Morante
+Mooe
+Montori
+Montminy
+Monteforte
+Montante
+Montanari
+Monsees
+Mondier
+Monden
+Monckton
+Monce
+Monarch
+Monarca
+Mompoint
+Mollema
+Molin
+Molima
+Molen
+Molash
+Moher
+Mogle
+Mogannam
+Moel
+Moehn
+Modesitt
+Mobilia
+Moag
+Miyagawa
+Mivshek
+Miu
+Mittman
+Mittleman
+Mittelsteadt
+Mittelstaedt
+Mitsch
+Mithell
+Miscione
+Mirbaha
+Mirabelli
+Mir
+Minon
+Minniti
+Minnerly
+Mingrone
+Minervini
+Minerd
+Minarcin
+Mimnaugh
+Milord
+Milnor
+Milnik
+Millers
+Milkowski
+Mikrot
+Mikles
+Miglorie
+Mientka
+Midthun
+Middlesworth
+Micklos
+Mickler
+Michetti
+Michelli
+Michelet
+Micallef
+Meyn
+Meullion
+Mette
+Metoxen
+Messore
+Messano
+Mesaros
+Mertel
+Merritts
+Merrion
+Merril
+Mermis
+Merlini
+Merker
+Meridith
+Mergel
+Merbaum
+Mente
+Mensi
+Menninger
+Mennen
+Menlove
+Menken
+Menezes
+Menette
+Mendyk
+Mendoca
+Mendivel
+Mendias
+Menasco
+Melloy
+Mellema
+Mellard
+Melis
+Meldahl
+Melberg
+Meirick
+Meinel
+Meiler
+Meile
+Meidl
+Meerdink
+Meer
+Medus
+Meduna
+Medovich
+Medine
+Medico
+Medici
+Mcvaigh
+Mctier
+Mcquirk
+Mcnight
+Mcmurrey
+Mcmurdo
+Mcmorries
+Mcmilleon
+Mcmickell
+Mcmicheal
+Mcmeel
+Mcleese
+Mclee
+Mclaws
+Mclanahan
+Mclaird
+Mckusker
+Mckibbens
+Mckenley
+Mckenize
+Mckendall
+Mckellop
+Mckellip
+Mckeirnan
+Mcinvale
+Mcguffee
+Mcgrue
+Mcgregory
+Mcgrann
+Mcgoey
+Mcglinn
+Mcgillicuddy
+Mcgillen
+Mcgeachy
+Mcgarrell
+Mcgannon
+Mcgalliard
+Mcfarlen
+Mcevers
+Mcerlean
+Mcennis
+Mcelvany
+Mcelvaine
+Mcdonal
+Mcdavitt
+Mccullick
+Mccrone
+Mccreadie
+Mccoun
+Mcconchie
+Mcconaughy
+Mcconahy
+Mcconaghy
+Mccomsey
+Mccoggle
+Mcclimans
+Mccleod
+Mccleaf
+Mcclafferty
+Mccatty
+Mccarry
+Mccance
+Mccament
+Mccaghren
+Mcbreen
+Mcardell
+Mcabier
+Mazell
+Mayotte
+Maybrier
+Mavis
+Mautone
+Matuszek
+Mattimoe
+Mattey
+Matterson
+Matten
+Matsushima
+Matsubara
+Matrone
+Matras
+Mato
+Matier
+Matheus
+Massucci
+Massoni
+Massare
+Maslin
+Mashaw
+Mase
+Mascola
+Masci
+Marze
+Marvray
+Marusak
+Martowski
+Martiny
+Martie
+Martabano
+Marsha
+Marschel
+Marsack
+Marsac
+Marohnic
+Markve
+Markis
+Marking
+Marken
+Marioni
+Marichalar
+Margosian
+Maretti
+Mardesich
+Marcussen
+Marchessault
+Marcey
+Maraldo
+Marafioti
+Manzanero
+Manwill
+Manual
+Manocchio
+Manko
+Manista
+Manire
+Manikowski
+Manganiello
+Manetta
+Mandy
+Mandino
+Mandarino
+Mancinelli
+Manasse
+Manary
+Manalang
+Malling
+Mallahan
+Maliska
+Malet
+Maleski
+Maldonaldo
+Malaterre
+Malaney
+Malagarie
+Malabe
+Maks
+Makinster
+Makar
+Maita
+Maiolo
+Mahley
+Magos
+Mago
+Magnotti
+Magnant
+Maglott
+Maglori
+Maenius
+Madkin
+Madarang
+Madagan
+Macrina
+Macquarrie
+Macphee
+Macneal
+Macmahon
+Maclellan
+Mackeen
+Maciver
+Machkovich
+Machan
+Macewen
+Macera
+Macer
+Maceachern
+Macdonell
+Macaskill
+Maaske
+Lysaght
+Lynum
+Lynema
+Lyas
+Lutton
+Luttman
+Lutsky
+Luthi
+Lutfy
+Lupoe
+Lundrigan
+Lunderville
+Lukan
+Luedeman
+Ludke
+Lucore
+Lucksinger
+Lucks
+Luckner
+Lucarell
+Lubelski
+Luarca
+Luaces
+Lozinski
+Loynes
+Lowis
+Lovorn
+Loverde
+Lovasz
+Loughery
+Lotzer
+Losito
+Loschiavo
+Lorsung
+Lorquet
+Lorkowski
+Lorino
+Lorey
+Lorente
+Loreman
+Lopaz
+Looft
+Lonie
+Longman
+Longhofer
+Longan
+Lomascolo
+Lomack
+Lolagne
+Lokaphone
+Logins
+Loggin
+Lofredo
+Loffler
+Loescher
+Loendorf
+Locus
+Lockyer
+Lockheart
+Lobendahn
+Lobasso
+Lob
+Lizana
+Livshits
+Litzau
+Litty
+Litteer
+Litsey
+Litrenta
+Litner
+Liszewski
+Lisman
+Lisboa
+Liquet
+Liptok
+Lineweaver
+Lindenpitz
+Lindel
+Lime
+Lillywhite
+Life
+Lievano
+Lieblong
+Liebler
+Lidey
+Libutti
+Liborio
+Libengood
+Leyson
+Leyland
+Lewczyk
+Lewark
+Leviner
+Levenstein
+Leuenberger
+Leszczynski
+Lestage
+Leske
+Lerwick
+Leray
+Lepkowski
+Leonor
+Lenyard
+Lenger
+Lendon
+Lemarie
+Leman
+Lelle
+Leisner
+Leisey
+Leischner
+Leimer
+Leigers
+Leiferman
+Leibfried
+Lehoullier
+Lehnortt
+Legget
+Legato
+Legath
+Legassie
+Legarreta
+Leftridge
+Leewright
+Ledsome
+Lecrone
+Lecourt
+Lecky
+Lechman
+Lebsack
+Lebouf
+Lebon
+Leazer
+Leavins
+Leadbeater
+Lawwill
+Lawall
+Lavorini
+Laviero
+Lavertue
+Lavalais
+Lautenbach
+Lausier
+Laurita
+Lauriano
+Laurange
+Launey
+Laughead
+Laufenberg
+Lauderman
+Laubhan
+Latunski
+Latulas
+Lastrape
+Lastiri
+Lason
+Laskoski
+Lasanta
+Laroux
+Larizza
+Larive
+Larish
+Laquerre
+Lappas
+Lapilio
+Lapadula
+Lapa
+Lanzi
+Lanzafame
+Lantier
+Lanski
+Laningham
+Langon
+Langdale
+Landron
+Landero
+Landauer
+Landacre
+Lamport
+Lamping
+Lamott
+Lamonda
+Lammi
+Lambiase
+Laite
+Lahaye
+Laframboise
+Lafone
+Laferte
+Laeger
+Ladieu
+Ladabouche
+Lachat
+Labonville
+Labbee
+Labatt
+Laban
+Kynaston
+Kwaterski
+Kuzniar
+Kuthe
+Kuter
+Kutchar
+Kurtin
+Kuramoto
+Kupstas
+Kuperman
+Kuns
+Kullmann
+Kuligowski
+Kukielka
+Kuehler
+Kudrna
+Kubie
+Kubera
+Kubas
+Kuba
+Kualii
+Krysinski
+Kryder
+Kronberger
+Kroft
+Kroencke
+Kristiansen
+Krigger
+Krieser
+Kretschman
+Krentz
+Krenke
+Kremers
+Kreitner
+Kreimer
+Kray
+Krawchuk
+Kravs
+Kranich
+Krampitz
+Kragh
+Krager
+Kozuch
+Kozloski
+Kozatek
+Kozakiewicz
+Kovalsky
+Kovalcik
+Kovack
+Kotera
+Kot
+Koszyk
+Kostel
+Kosmicki
+Koshy
+Korona
+Koroma
+Korba
+Koopmann
+Konstantinidi
+Kolodzik
+Kolodzieski
+Kolle
+Kolkmann
+Kolker
+Kolda
+Kokaly
+Kofford
+Koepper
+Koeing
+Koehnen
+Kodish
+Kodani
+Kocur
+Kocourek
+Kobza
+Koble
+Koback
+Knutzen
+Knows
+Knolton
+Knoblauch
+Knispel
+Knieper
+Knepshield
+Klyce
+Klunk
+Kluka
+Klostermann
+Klosinski
+Klish
+Klint
+Klinner
+Klindt
+Klimko
+Klicker
+Kleman
+Kleinsorge
+Kleinfelder
+Kleier
+Klas
+Klaman
+Kizzee
+Kitto
+Kitka
+Kirtdoll
+Kirscht
+Kintzer
+Kinstle
+Kinning
+Kinniburgh
+Kinnett
+Kinker
+Kinkelaar
+Kings
+Kingham
+Kingfisher
+Kimmet
+Killingbeck
+Kilberg
+Kikuchi
+Kikkert
+Kiesow
+Kienitz
+Kidner
+Kida
+Kid
+Khuu
+Khatak
+Khaleck
+Kezar
+Keyton
+Ketelhut
+Kesley
+Keshishyan
+Kerzman
+Kertesz
+Kerslake
+Kerscher
+Kernes
+Kerin
+Ker
+Kenimer
+Kenfield
+Kempe
+Kemick
+Kem
+Keitsock
+Keisker
+Keery
+Keblish
+Kebalka
+Kearny
+Kearby
+Kayler
+Kavin
+Kauer
+Kattan
+Katoa
+Kassis
+Kashuba
+Kashan
+Kartman
+Karry
+Karpel
+Karo
+Karnopp
+Karmazyn
+Karjala
+Karcz
+Karasti
+Karagiannis
+Kapoi
+Kapanke
+Kanz
+Kaniewski
+Kanemoto
+Kaneholani
+Kandt
+Kampfer
+Kammann
+Kamler
+Kamal
+Kalvig
+Kalmen
+Kalmar
+Kallstrom
+Kallin
+Kallbrier
+Kakaviatos
+Kakar
+Kahahane
+Kagel
+Kabat
+Kabanuck
+Kaas
+Jurczak
+Jurasin
+Juras
+Junke
+Junghans
+Jungen
+Jund
+Juliusson
+Juhnke
+Juett
+Jolla
+Jokinen
+Jokela
+Joffe
+Joecks
+Jochumsen
+Joa
+Jeziorski
+Jesseman
+Jessamy
+Jernejcic
+Jergenson
+Jerdon
+Jensrud
+Jellinek
+Jedrey
+Jedele
+Jeannette
+Jauron
+Jatho
+Jarrel
+Januszewski
+Janski
+Janovsek
+Janning
+Janikowski
+Jane
+Jandres
+Jamaica
+Jalonen
+Jainlett
+Jahnsen
+Jahde
+Jagow
+Jagielski
+Jaffray
+Jaecks
+Jacquot
+Jacoway
+Jacocks
+Iwami
+Isadore
+Irmeger
+Irie
+Iredale
+Iqbal
+Inscoe
+Inklebarger
+Ingemi
+Immen
+Imig
+Imberg
+Imamura
+Illies
+Ilacqua
+Ijams
+Iha
+Iden
+Ibraham
+Ibey
+Ialongo
+Iafrate
+Hyzer
+Hyacinthe
+Huyard
+Huxman
+Hutchkiss
+Hutchingson
+Husson
+Hussman
+Hurm
+Hupka
+Hunyadi
+Hunstad
+Humpert
+Hummons
+Hultz
+Hulton
+Hules
+Huisenga
+Huhta
+Hugueley
+Hughe
+Huggler
+Hufton
+Huffstickler
+Huddelston
+Huba
+Hrivnak
+Hoysradt
+Howorth
+Howenstine
+Hovda
+Hourani
+Houglum
+Houch
+Hotalen
+Hosse
+Horwich
+Horvitz
+Horoschak
+Hornor
+Hornbrook
+Horita
+Hoque
+Hopman
+Hoovler
+Hoople
+Hookfin
+Honeysucker
+Honeycut
+Honerkamp
+Homyak
+Homa
+Holzwart
+Holzerland
+Holyoke
+Holtry
+Holterman
+Holohan
+Hollinshed
+Hollington
+Hollenshead
+Holey
+Holderby
+Holak
+Hokkanen
+Hohner
+Hogsed
+Hoglen
+Hogen
+Hogberg
+Hofland
+Hofius
+Hoffis
+Hofferber
+Hoffarth
+Hofacker
+Hoekman
+Hodor
+Hochstetter
+Hochnadel
+Hobbins
+Hoa
+Hlavaty
+Hittner
+Hitson
+Hirtz
+Hirschi
+Hinkes
+Hinke
+Hindley
+Hince
+Hilse
+Hilke
+Hilferty
+Hildesheim
+Hikes
+Hignite
+Higman
+Hiemer
+Hidden
+Hickinbotham
+Hewatt
+Hetz
+Hetsler
+Hessian
+Hershaw
+Herra
+Hernander
+Herlocker
+Hepper
+Henseler
+Henri
+Hennick
+Hennecke
+Hendrikson
+Henderlight
+Hellstrom
+Helderman
+Heitland
+Heistand
+Heiskell
+Heisinger
+Heiserman
+Heinritz
+Heinly
+Heinlen
+Heimerdinger
+Heimbigner
+Heidbreder
+Hegwer
+Hedeen
+Hebrank
+Heberlein
+Heaslet
+Hearin
+Hazle
+Hazelbush
+Hayzlett
+Hayre
+Haymans
+Hayenga
+Hayduk
+Haward
+Havner
+Haushalter
+Hauf
+Hatke
+Hatchel
+Hassard
+Haskovec
+Hashmi
+Harvest
+Harvath
+Hartill
+Harteau
+Harshfield
+Harrigill
+Harriet
+Haros
+Haroldson
+Harmeson
+Harl
+Harkley
+Hariston
+Harington
+Harian
+Hargus
+Hargens
+Hardina
+Haraldson
+Harajly
+Hapke
+Hapeman
+Hanz
+Hanthorn
+Hanry
+Hannen
+Hannasch
+Hannam
+Hanifan
+Hanft
+Handon
+Handford
+Hancher
+Hancey
+Hample
+Hammrich
+Hammerstrom
+Hambric
+Halwick
+Halma
+Hallgren
+Hallet
+Hallada
+Halla
+Halik
+Halgas
+Halcon
+Halbrooks
+Hakel
+Hairfield
+Hainesworth
+Haggarty
+Hagenhoff
+Hagebusch
+Hagadone
+Haft
+Haflett
+Haefele
+Haddow
+Hackbart
+Haberer
+Haass
+Gwinner
+Gwathney
+Gwartney
+Gutterrez
+Gutoski
+Gutkin
+Gutherie
+Gutches
+Gustus
+Gustison
+Gustaveson
+Gurtner
+Gurkin
+Gummo
+Gulliksen
+Gulke
+Guldin
+Gulden
+Guitierez
+Guile
+Guildford
+Guidice
+Gugerty
+Guffy
+Gueningsman
+Gudgell
+Guderjahn
+Guastella
+Guariglia
+Guardia
+Gryniuk
+Grueser
+Grudem
+Growden
+Grossett
+Gropper
+Gron
+Grodin
+Groch
+Grismore
+Gripper
+Grinvalsky
+Grima
+Griffth
+Griess
+Greynolds
+Gresh
+Greminger
+Gregoria
+Greenwade
+Greenlief
+Greenier
+Grayes
+Gravell
+Grassmyer
+Grappe
+Grantland
+Grandin
+Grandel
+Grandbois
+Granahan
+Gramham
+Graffeo
+Graeter
+Gradwell
+Gradel
+Grabo
+Graban
+Goy
+Govoni
+Governale
+Govern
+Gouty
+Goughnour
+Goude
+Goubeaux
+Goth
+Gosline
+Goslee
+Goshen
+Gosewisch
+Gorzynski
+Gortman
+Gorter
+Gordin
+Gord
+Goos
+Goodwine
+Goodrick
+Goodley
+Gombert
+Goletz
+Goldy
+Goldthwaite
+Goldthwait
+Goldizen
+Golar
+Goist
+Gofman
+Goffer
+Goerges
+Goeltz
+Goedicke
+Goedecke
+Godnick
+Gocke
+Goade
+Gneiser
+Gluth
+Glovier
+Glomski
+Glodo
+Gloden
+Glenister
+Glawson
+Glasier
+Gladysz
+Gladstein
+Gjertsen
+Giudice
+Gitto
+Gittelman
+Girvin
+Girolamo
+Gionfriddo
+Gingell
+Gimble
+Gilhousen
+Gilboy
+Gilberti
+Gigantino
+Gietzen
+Gieseking
+Gianikas
+Ghosn
+Ghosh
+Geyman
+Gevara
+Getsinger
+Gessert
+Gerrits
+Gerrior
+Geris
+Gerhauser
+Gerety
+Genzone
+Genuario
+Gentles
+Gentille
+Genter
+Genetti
+Gelle
+Gelfand
+Gelabert
+Gekas
+Geck
+Gearin
+Gdovin
+Gaydosh
+Gawith
+Gave
+Gauntlett
+Gaugler
+Gaudy
+Gaub
+Gatten
+Gathje
+Gasperini
+Gasner
+Gasco
+Gascho
+Gasbarro
+Garvis
+Garra
+Garnette
+Garing
+Garick
+Gardunio
+Gardon
+Gardemal
+Garde
+Garczynski
+Garant
+Ganus
+Gantnier
+Ganis
+Gangloff
+Gangler
+Ganer
+Ganem
+Gandolfo
+Gampp
+Gallihugh
+Galletti
+Gallenstein
+Gallarello
+Galla
+Galka
+Galayda
+Galarneau
+Galapon
+Gaito
+Gaglione
+Gady
+Gadsen
+Gachupin
+Gaboury
+Futterman
+Fusch
+Furuta
+Furth
+Furber
+Fune
+Funai
+Fuess
+Frutchey
+Frumkin
+Fruhling
+Frommer
+Fromdahl
+Froehner
+Frizzle
+Friends
+Friederich
+Freyre
+Freilich
+Fregia
+Frediani
+Frederico
+Frater
+Fraile
+Foste
+Fosselman
+Fosnaugh
+Fosburg
+Fortis
+Fortgang
+Forstner
+Forson
+Forseth
+Forkin
+Forister
+Forinash
+Footer
+Fontillas
+Fontenelle
+Fonesca
+Folker
+Fogerson
+Fogelquist
+Flye
+Flummer
+Floth
+Floro
+Florine
+Flies
+Flexer
+Flessner
+Flatness
+Flank
+Fland
+Flahive
+Flager
+Fiveash
+Fitzner
+Fitzke
+Fitcheard
+Fisherman
+Fishbeck
+Fipps
+Fiorino
+Finster
+Finken
+Finigan
+Fingal
+Finer
+Filsaime
+Fillingim
+Filipponi
+Fila
+Fies
+Fiebelkorn
+Fiducia
+Fiallo
+Fetherston
+Fetherolf
+Fesmire
+Fesenmyer
+Ferroni
+Ferriss
+Ferrini
+Ferrick
+Ferraris
+Ferniza
+Fernades
+Ferdig
+Ferandez
+Feoli
+Fenninger
+Fenney
+Femi
+Fejes
+Fehlman
+Feger
+Fede
+Febo
+Febbraio
+Feasel
+Feagley
+Fayad
+Favaloro
+Fauerbach
+Fauble
+Fasheh
+Farrant
+Farra
+Faro
+Farinacci
+Farfaglia
+Farell
+Farb
+Farace
+Fanjoy
+Fangmann
+Famulare
+Falsetta
+Fallows
+Fallert
+Falero
+Faldyn
+Falconi
+Falce
+Fait
+Fairburn
+Faiola
+Faiella
+Fahlsing
+Faggett
+Fafinski
+Fadness
+Fabros
+Fabert
+Everidge
+Evaristo
+Eustache
+Etzkorn
+Etier
+Estabillo
+Esquivias
+Esquirel
+Eslava
+Eschete
+Esau
+Erway
+Ertzbischoff
+Eron
+Erner
+Ermitano
+Ermitanio
+Ermert
+Erie
+Erdley
+Equihua
+Enzor
+Ensing
+Enns
+Engleking
+Engelkes
+Endlich
+Endler
+Emry
+Emms
+Emmerling
+Emerich
+Ellsbury
+Ellie
+Elizarraras
+Eliot
+Eliopoulos
+Elery
+Elek
+Elderidge
+Elbaum
+Ekins
+Ekin
+Eisley
+Eilderts
+Eikleberry
+Eigo
+Eighmy
+Eichel
+Ehly
+Egloff
+Egland
+Eggington
+Eggenberger
+Egar
+Egans
+Eftekhari
+Efford
+Eeds
+Edvalson
+Edin
+Edgman
+Edemann
+Edelmann
+Eddens
+Eckl
+Eckerle
+Eckelman
+Ebrahim
+Eberth
+Eberspacher
+Ebbighausen
+Ebaugh
+Easly
+Eash
+Dzledzic
+Dyett
+Dyba
+Dworaczyk
+Duttry
+Duthie
+Duszynski
+Duso
+Dushaj
+Dusett
+Dus
+Durman
+Durkins
+Durick
+Duplechain
+Dunnivan
+Dunlow
+Dunivan
+Dumars
+Dumaine
+Duliba
+Dulany
+Duka
+Duft
+Dufrane
+Duffek
+Duellman
+Ducking
+Dubourg
+Drzewiecki
+Drugan
+Drozdowski
+Drozda
+Dronet
+Drilling
+Driesenga
+Dreyfuss
+Drevs
+Dreben
+Draudt
+Draleau
+Dragos
+Draghi
+Doyer
+Dowlin
+Douma
+Dotterweich
+Dottavio
+Doroff
+Dornon
+Dorland
+Doop
+Donndelinger
+Donehoo
+Donate
+Donado
+Dommer
+Dominici
+Domann
+Dolio
+Dolence
+Doland
+Dolak
+Doersam
+Doerrer
+Doede
+Dockham
+Dobrich
+Dobosz
+Dobin
+Dobbratz
+Divlio
+Divel
+Ditzel
+Disalvatore
+Diotte
+Dinnen
+Dinkin
+Dimler
+Dimiceli
+Dimeglio
+Dimascio
+Dimare
+Diluca
+Dilsaver
+Dillen
+Dilibero
+Dile
+Digioia
+Difede
+Diefenbach
+Diedrick
+Dickmann
+Dickes
+Dickason
+Dicapua
+Dicaprio
+Dibrell
+Dibley
+Dibattista
+Deyon
+Devotie
+Devoid
+Deval
+Detlefsen
+Destro
+Destiche
+Desposito
+Desola
+Deshotels
+Descombes
+Deschepper
+Desautel
+Desano
+Deroy
+Derosset
+Derosby
+Deroeck
+Derocher
+Dergance
+Deren
+Deptula
+Deprey
+Depolis
+Depner
+Depetro
+Denunzio
+Densford
+Dennington
+Dene
+Dender
+Denbo
+Demuro
+Demoranville
+Demling
+Demerson
+Demelis
+Demeglio
+Dembo
+Demattia
+Demarinis
+Delprincipe
+Deloria
+Delnoce
+Delmedico
+Dellow
+Delles
+Dellavalle
+Dellamora
+Delguidice
+Delgato
+Delfs
+Delcourt
+Delcolle
+Delbert
+Delaportilla
+Delahoz
+Delacueva
+Deisch
+Deike
+Degro
+Degonia
+Degollado
+Degolier
+Degirolamo
+Degener
+Degele
+Degeest
+Degeare
+Defina
+Defabio
+Deeley
+Decraene
+Decou
+Decorte
+Declercq
+Decinti
+Dechambeau
+Debutts
+Debro
+Deblieck
+Deblasi
+Debem
+Deavila
+Deases
+Deangeles
+Deahl
+Daymude
+Daven
+Datil
+Daros
+Darnick
+Darienzo
+Dardy
+Daponte
+Dannhaus
+Danneman
+Danielle
+Dani
+Danger
+Dangel
+Danes
+Danekas
+Dandrow
+Dambrose
+Dalpe
+Dalesandro
+Daiton
+Dainels
+Daigh
+Dahnke
+Dahme
+Dahling
+Dagata
+Dack
+Czaplicki
+Czachorowski
+Cuttitta
+Cutaia
+Custance
+Curless
+Curie
+Curi
+Cupelli
+Cumens
+Cumbass
+Cumba
+Cullars
+Cullar
+Cukaj
+Cubito
+Cuascut
+Crytzer
+Crye
+Cruzen
+Cruser
+Crunkleton
+Crummett
+Crumbliss
+Cropley
+Cronquist
+Cronkite
+Cronic
+Crombie
+Crockwell
+Crnkovich
+Critcher
+Cristo
+Cristales
+Crisanti
+Crier
+Cretsinger
+Crest
+Creson
+Crelia
+Crecco
+Craze
+Craveiro
+Cratch
+Crapps
+Cran
+Craigmiles
+Craiger
+Craige
+Crady
+Cradic
+Craddieth
+Cowels
+Coveney
+Courcy
+Coulbourne
+Cotsis
+Cotrone
+Cotney
+Cotilla
+Costaneda
+Costabile
+Cossel
+Cossa
+Cos
+Corte
+Corsino
+Corria
+Cornog
+Cornely
+Corio
+Corino
+Corington
+Coressel
+Cordone
+Corbisiero
+Corbelli
+Copps
+Coovert
+Coopwood
+Cooner
+Cookman
+Conzales
+Conver
+Contratto
+Conrady
+Conradi
+Connel
+Conneely
+Conmy
+Comunale
+Comber
+Comans
+Colvert
+Columbo
+Coluccio
+Colp
+Colop
+Collini
+College
+Colestock
+Colebank
+Colasante
+Colasacco
+Colapietro
+Cokeley
+Coia
+Cocuzza
+Coalson
+Co
+Clowes
+Cliche
+Clevette
+Cleven
+Clerico
+Clearwater
+Civiello
+Ciullo
+Citro
+Cirocco
+Cioppa
+Cilek
+Cieszynski
+Cieri
+Cicerchia
+Ciaschi
+Ciani
+Cianchetti
+Chudy
+Chuc
+Chryst
+Christodoulou
+Christin
+Chrisley
+Chokshi
+Chmela
+Chkouri
+Chiodini
+Chio
+Chimilio
+Chilen
+Chilek
+Childrey
+Chier
+Chicas
+Chiaro
+Chiappone
+Chiappinelli
+Chiado
+Chhom
+Chesterfield
+Chesteen
+Cheshier
+Cherrez
+Cherep
+Chene
+Cheevers
+Checkett
+Cheaney
+Chayka
+Chawla
+Chasin
+Chasen
+Charvat
+Char
+Chapoton
+Chantos
+Chantler
+Chant
+Chadez
+Chad
+Chaco
+Chabez
+Cerrito
+Ceppetelli
+Centanni
+Celso
+Cederberg
+Cedar
+Cecchetti
+Cavel
+Cavanah
+Cavagna
+Catus
+Catton
+Catterton
+Catrambone
+Catherwood
+Catherman
+Cataldi
+Castellana
+Castellan
+Cassey
+Casparis
+Casilla
+Cashdollar
+Casaceli
+Carvana
+Carriedo
+Carrecter
+Carraher
+Carrabine
+Carpinelli
+Carouthers
+Carnovale
+Carmany
+Carles
+Caretto
+Careaga
+Cardosa
+Cardelli
+Carbine
+Carathers
+Caraker
+Caracci
+Capuchin
+Cappelletti
+Capistran
+Capdeville
+Caparros
+Canute
+Cante
+Canizares
+Canel
+Canclini
+Cancino
+Campus
+Campise
+Campen
+Cammarano
+Camilli
+Camic
+Camey
+Calwell
+Calvey
+Calvary
+Callo
+Callinan
+Callais
+Calizo
+Calixto
+Calisto
+Calip
+Calibuso
+Caira
+Cahillane
+Cahalane
+Cahal
+Caffery
+Caffarelli
+Cafarelli
+Cadlett
+Cacciatori
+Cabebe
+Byus
+Byrnside
+Byrer
+Byone
+Buza
+Buttrum
+Buttel
+Butremovic
+Butanda
+Bustin
+Bussen
+Bushlen
+Bushart
+Burtchell
+Burrel
+Burnard
+Burlett
+Burkeen
+Burce
+Buote
+Bunyan
+Buntrock
+Bunck
+Bumpas
+Bulleri
+Buglione
+Bugge
+Bueter
+Buerk
+Buenger
+Buehrle
+Buechele
+Budrow
+Buddenhagen
+Bucolo
+Buchenau
+Bucco
+Buccino
+Bubar
+Bruzas
+Brutsch
+Bruschke
+Brunot
+Brungard
+Brund
+Bruender
+Brucks
+Bruchey
+Brozowski
+Brownd
+Brothern
+Broomhead
+Bronw
+Brom
+Brog
+Brodigan
+Brockhaus
+Brockel
+Broadaway
+Brletich
+Briston
+Brissett
+Brines
+Brillon
+Brilliant
+Brightbill
+Brigges
+Briel
+Bresciani
+Brents
+Breitmeyer
+Breithaupt
+Breidenthal
+Breden
+Bredemeier
+Breckinridge
+Brecheisen
+Brecheen
+Breazeal
+Bream
+Brazzel
+Brawdy
+Brave
+Brashers
+Branz
+Branyon
+Brantz
+Brannam
+Brankovich
+Brandle
+Branchaud
+Branca
+Bramley
+Bramante
+Bramall
+Brakeman
+Bradby
+Bozzo
+Bozelle
+Boyarski
+Bowline
+Bowey
+Bowerize
+Bowdon
+Bowdler
+Boutros
+Bouten
+Bourdier
+Bouras
+Boufford
+Bottex
+Bottemiller
+Bothman
+Botcher
+Boshers
+Borris
+Bornemann
+Bonus
+Bonnot
+Bonifant
+Bongiardina
+Bonenberger
+Bonasera
+Bollier
+Bolar
+Bokman
+Bokanovich
+Boissonnault
+Boiles
+Bohrn
+Bohlke
+Bogenschutz
+Bogel
+Bogda
+Boevers
+Boever
+Boender
+Boehringer
+Boehne
+Bodor
+Bodda
+Bodak
+Bocker
+Bockenkamp
+Boche
+Blyden
+Bluto
+Bludworth
+Bloxsom
+Blomstrom
+Bloise
+Bloebaum
+Blier
+Bleiweiss
+Blegen
+Bleacher
+Blaum
+Blasz
+Blasingim
+Blasengame
+Blanda
+Blagman
+Blackstad
+Blackham
+Blache
+Bixel
+Bitters
+Bissegger
+Bisker
+Bishoff
+Bisard
+Bis
+Birtwell
+Birley
+Birkenmeier
+Birkenholz
+Birkeland
+Birdsey
+Birdo
+Birdinground
+Binner
+Bilsborough
+Billot
+Billops
+Billingham
+Bigney
+Bigg
+Bienkowski
+Bienek
+Bielefeld
+Bielec
+Biddie
+Bickell
+Bichler
+Bibo
+Biava
+Biagi
+Biagas
+Bhayani
+Bez
+Beyene
+Beyda
+Bevels
+Bettner
+Bettinson
+Betson
+Beto
+Bessix
+Bessire
+Bertschy
+Bertozzi
+Bertoncini
+Bertelson
+Berteau
+Berrong
+Berrones
+Berringer
+Berrigan
+Bernsen
+Berlingeri
+Berken
+Berka
+Berges
+Bergdorf
+Bergara
+Bergant
+Bergamini
+Beren
+Berdugo
+Berdine
+Berberian
+Benvenuti
+Benish
+Benincase
+Benek
+Benedith
+Bendas
+Benak
+Bena
+Beltrame
+Belsheim
+Belotti
+Bellrichard
+Belleville
+Beliles
+Belgrade
+Belcastro
+Bekius
+Bekhit
+Beightol
+Behel
+Beetz
+Bedson
+Becze
+Beckmeyer
+Beckey
+Beckers
+Beckelhimer
+Beccue
+Beberwyk
+Bebber
+Beamesderfer
+Beacom
+Bazzle
+Bazil
+Baynham
+Bayhonan
+Bayas
+Bawany
+Bava
+Baumgardt
+Bauerkemper
+Baudry
+Baudino
+Battko
+Battisti
+Batta
+Bassano
+Baskas
+Baseler
+Basanta
+Bartucci
+Bartron
+Barthold
+Bartamian
+Barsalou
+Barrineau
+Barriger
+Barreneche
+Barkie
+Barich
+Bardes
+Barbano
+Baral
+Baragar
+Baque
+Banther
+Banome
+Bannowsky
+Banke
+Baniaga
+Bandley
+Banahan
+Banaag
+Bamba
+Baltzer
+Balster
+Balnis
+Balkin
+Bali
+Balfe
+Balerio
+Balent
+Baldyga
+Baldor
+Baldinger
+Baldassano
+Baldacci
+Balanoff
+Balado
+Balaban
+Balaam
+Bakes
+Bajwa
+Baisch
+Bahnsen
+Bahls
+Bahler
+Bahamonde
+Bagdasarian
+Bagaoisan
+Bafia
+Baese
+Badolato
+Bado
+Badder
+Bacurin
+Backers
+Bachor
+Babe
+Babbit
+Babauta
+Baadsgaard
+Azzara
+Azebedo
+Avril
+Avello
+Aveline
+Authur
+Ausby
+Auricchio
+Auna
+Aukerman
+Auckerman
+Auck
+Auble
+Atterson
+Attard
+Aswegan
+Aste
+Asta
+Assaf
+Aspen
+Asken
+Asif
+Asiedu
+Ashner
+Asel
+Aschenbach
+Arvay
+Arvan
+Artus
+Artley
+Arrollo
+Aroyo
+Aronov
+Aromin
+Arnsworth
+Arnspiger
+Arnn
+Armant
+Arington
+Argubright
+Arentz
+Arcoraci
+Arbuthnot
+Arbo
+Aquilina
+Aquilera
+Apt
+Apsey
+Appolonia
+Apollo
+Apana
+Antista
+Anshutz
+Anon
+Anno
+Annala
+Anklam
+Angold
+Angelone
+Angeline
+Angeletti
+Andren
+Andreadis
+Andera
+Andelman
+Andel
+Anctil
+Anchors
+Anacker
+Ampy
+Amons
+Amirault
+Amir
+Amezaga
+Ameigh
+Alyea
+Altvater
+Altig
+Altermatt
+Alo
+Almengor
+Alme
+Allvin
+Allocco
+Allegrini
+Aliment
+Algee
+Alexanian
+Aler
+Aldo
+Albero
+Alarid
+Akiona
+Akemon
+Ajello
+Aitcheson
+Ainley
+Ailey
+Ahluwalia
+Ahlf
+Ahlbrecht
+Agundez
+Agro
+Agins
+Aggarwal
+Afalava
+Adriano
+Adomaitis
+Adolphus
+Adlam
+Adie
+Adey
+Adduci
+Addleman
+Adamyan
+Acothley
+Acklen
+Ackert
+Ackerly
+Acencio
+Accosta
+Abundiz
+Abedi
+Abbassi
+Abbasi
+Aanerud
+Aakre
+Aagaard
+Zwickl
+Zuver
+Zurasky
+Zumbo
+Zumba
+Zuckerwar
+Zuccarelli
+Zubris
+Zoucha
+Zorns
+Zorc
+Zitzow
+Zitzloff
+Zirkles
+Zippe
+Ziola
+Zinz
+Zinsmeister
+Zincke
+Zieschang
+Zierdt
+Zien
+Ziemke
+Zidek
+Zickler
+Zeuner
+Zerba
+Zera
+Zenger
+Zeltmann
+Zelle
+Zelinka
+Zelek
+Zele
+Zeiner
+Zeimet
+Zeidler
+Zecchini
+Zebley
+Zdanowicz
+Zbell
+Zaro
+Zaremski
+Zar
+Zani
+Zancanella
+Zana
+Zambarano
+Zakar
+Zadorozny
+Zader
+Zaccaro
+Ysquierdo
+Yoxall
+Youst
+Youngstrom
+Youn
+Youker
+Yoss
+Yoshina
+Yonke
+Yonemura
+Yohannes
+Yock
+Yerhot
+Yengo
+Yehle
+Yanofsky
+Yaker
+Yagues
+Yach
+Ya
+Xue
+Wyrosdick
+Wygle
+Wygand
+Wurzer
+Wurl
+Wunderlin
+Wunderle
+Wuerth
+Writer
+Wrighten
+Wrich
+Wozny
+Wozney
+Wowk
+Wouters
+Wormington
+Worf
+Woolem
+Woodrich
+Wooderson
+Wonder
+Womeldorf
+Wolz
+Woltmann
+Wolstenholme
+Wollmuth
+Wolle
+Wolfard
+Woldridge
+Wojtanowski
+Wojner
+Woitowitz
+Woehl
+Wittenburg
+Wittel
+Witschi
+Witaszek
+Witaker
+Wiszynski
+Wiswall
+Wiss
+Wisher
+Wisenbaker
+Wires
+Winsky
+Winfough
+Windler
+Winckler
+Wimes
+Wiltberger
+Wilm
+Willrich
+Willoby
+Willimon
+Willenborg
+Wilda
+Wilczewski
+Wilcock
+Wiggens
+Wigboldy
+Wiesler
+Wies
+Wienhoff
+Wielgus
+Wiebers
+Wieber
+Wickizer
+Wichrowski
+Wibbens
+Whyard
+Wholey
+Whitsey
+Whitlingum
+Whitlach
+Whirry
+Wharry
+Wharff
+Whack
+Weyman
+Weyler
+Wethje
+Westveer
+Westmorland
+Westerhold
+Wesselman
+Wesloh
+Wery
+Wermers
+Werlinger
+Werksman
+Wenzinger
+Weninger
+Wendeln
+Wendelin
+Wenck
+Wember
+Welters
+Welland
+Welchman
+Welchel
+Weitnauer
+Weissler
+Weinger
+Weimann
+Weigert
+Weidert
+Wehby
+Wehbe
+Weck
+Wechter
+Weaving
+Weather
+Weal
+Weagle
+Wdowiak
+Wayns
+Waycott
+Waychoff
+Waterfall
+Watcher
+Watahomigie
+Wasowski
+Wasner
+Washko
+Washing
+Washell
+Wartenberg
+Warson
+Warrenfeltz
+Warp
+Warmbrodt
+Warhurst
+Wardsworth
+Wanzek
+Wanta
+Wansing
+Wankel
+Wangberg
+Wanberg
+Wamack
+Waltzer
+Walthers
+Walterson
+Walshe
+Walrond
+Wallschlaeger
+Wallgren
+Walema
+Waldram
+Waldhauser
+Waldecker
+Walby
+Wakin
+Wakabayashi
+Wah
+Wagy
+Waggner
+Wagenaar
+Wage
+Waffle
+Wadzinski
+Wademan
+Wackerly
+Wachs
+Wable
+Vredenburg
+Vrana
+Vrable
+Voyer
+Voto
+Vosper
+Vosberg
+Vorhees
+Voran
+Vora
+Vonstein
+Vondoloski
+Voltin
+Volpicelli
+Volland
+Volentine
+Volcko
+Vojtko
+Voice
+Vogeler
+Vizzini
+Vizena
+Vix
+Vitko
+Viste
+Visor
+Visco
+Virock
+Vinup
+Vinion
+Vincenzo
+Villas
+Villarta
+Villari
+Vilello
+Vigne
+Viener
+Vielmas
+Vielhauer
+Viehman
+Vidulich
+Vidinha
+Videen
+Vickerson
+Vicker
+Vertz
+Verry
+Vermeesch
+Verhulst
+Verhoff
+Verhagen
+Verhaeghe
+Vergo
+Vergeer
+Verdino
+Venus
+Ventrella
+Ventola
+Venter
+Vennes
+Venneri
+Venditto
+Velzy
+Velilla
+Velie
+Velandia
+Vecker
+Vecellio
+Vear
+Vavricka
+Vautrin
+Vates
+Vassall
+Vasmadjides
+Varty
+Varriano
+Varriale
+Varrato
+Varnedoe
+Varillas
+Vardaman
+Varajas
+Vaquero
+Vanzyl
+Vanvleet
+Vanvleck
+Vansoest
+Vanskiver
+Vanskike
+Vanruler
+Vanputten
+Vanoy
+Vanous
+Vanoort
+Vanliew
+Vanlew
+Vanhulle
+Vanhoozier
+Vanhofwegen
+Vanhaitsma
+Vanecek
+Vandrunen
+Vandixon
+Vandivier
+Vandiford
+Vandezande
+Vandewege
+Vanderzanden
+Vanderwerff
+Vanderwerf
+Vanderschel
+Vandergiessen
+Vandenberghe
+Vandehei
+Vandee
+Vancheri
+Vanbramer
+Valsin
+Valli
+Valido
+Valenzano
+Vajda
+Vaillencourt
+Vacheresse
+Va
+Uzdygan
+Uyetake
+Usilton
+Urueta
+Ursprung
+Ursiak
+Urquilla
+Urquidi
+Urfer
+Ureta
+Urbancic
+Ura
+Upwall
+Uptegrove
+Uphaus
+Upadhyaya
+Unterburger
+Unch
+Unavailable
+Unangst
+Umphenour
+Umbenhauer
+Ulseth
+Ulatowski
+Ukosata
+Uhyrek
+Uhrmacher
+Uhlich
+Ueno
+Uelmen
+Udoh
+Ude
+Uchytil
+Tzeng
+Typhair
+Twelves
+Twehous
+Tuxhorn
+Turybury
+Turro
+Turne
+Turnblom
+Turkus
+Turks
+Turbin
+Turbes
+Tunick
+Tumpkin
+Tuholski
+Tuggie
+Tufnell
+Tubertini
+Tubaugh
+Tsutsui
+Tsuha
+Tsuda
+Tsinnie
+Trupp
+Trupiano
+Trupia
+Truner
+Trundle
+Trumm
+Trullinger
+Truell
+Trucco
+Trowers
+Trover
+Trosien
+Tronnes
+Trompeter
+Tromp
+Trolio
+Troendle
+Trobaugh
+Triska
+Trimarco
+Trifiletti
+Tridle
+Tricoche
+Tresvant
+Trest
+Tresler
+Tresca
+Tremont
+Tremayne
+Treinen
+Treichler
+Treglia
+Treamer
+Traxson
+Traugh
+Trasher
+Trapasso
+Trant
+Trancoso
+Traister
+Trailor
+Trageser
+Traficante
+Trac
+Toya
+Towson
+Tovrea
+Totherow
+Tote
+Tortorelli
+Torri
+Tornabene
+Torigian
+Torello
+Toppa
+Topor
+Toothill
+Toop
+Tonsil
+Tomsich
+Tommie
+Tomlison
+Tolmich
+Tollner
+Tollefsrud
+Toledano
+Tolayo
+Toenges
+Toefield
+Tock
+Tobiasz
+Tobery
+Tobert
+Toban
+Toback
+Tjarks
+Tiznado
+Titlow
+Tishler
+Tirabassi
+Tippet
+Tinkey
+Timson
+Timperman
+Timmis
+Timmermans
+Timme
+Timberman
+Tikkanen
+Tietze
+Tierman
+Tiberi
+Thuringer
+Thul
+Thu
+Thro
+Thornwell
+Thomlison
+Thomlinson
+Thomassen
+Thimmes
+Thilking
+Thierman
+Thielemann
+Thiboutot
+Thibideau
+Theresa
+Theard
+Thavichith
+Thaut
+Tezak
+Tetzloff
+Teto
+Tetlow
+Tessler
+Tesseyman
+Teskey
+Tes
+Terzian
+Terwillegar
+Tervo
+Terronez
+Ternasky
+Termini
+Terboss
+Teramoto
+Tepley
+Tenuta
+Tenen
+Tellio
+Tellefson
+Telecky
+Tekell
+Tefertiller
+Teece
+Tedesko
+Tederous
+Tebeau
+Tear
+Teahan
+Tazewell
+Tazelaar
+Tavano
+Tatsapaugh
+Tatlock
+Tataris
+Tassinari
+Tassie
+Tarvis
+Tarkey
+Tarangelo
+Tappa
+Tanna
+Tanikella
+Tamblyn
+Tamaro
+Talyor
+Tallas
+Talayumptewa
+Talaska
+Taj
+Tagliarini
+Tagata
+Taflinger
+Taddonio
+Tacderan
+Tablang
+Tabisula
+Tabicas
+Tabar
+Szwed
+Szumski
+Szumigala
+Szollosi
+Szczesny
+Sypniewski
+Syon
+Sylvan
+Syal
+Swor
+Swoopes
+Swoap
+Swire
+Swimmer
+Swiler
+Swida
+Sweezer
+Sweep
+Sweeley
+Swede
+Swearengen
+Sweadner
+Swartzwelder
+Swanhart
+Sveen
+Svay
+Sutyak
+Sutten
+Sutler
+Suski
+Surprise
+Supernault
+Suozzo
+Suns
+Sunder
+Sumney
+Summarell
+Sumera
+Sulzbach
+Sulfridge
+Sukhram
+Suk
+Suitor
+Sughroue
+Sugahara
+Sudlow
+Sudan
+Sudak
+Subido
+Style
+Stweart
+Sturz
+Sturdy
+Sturchio
+Stulce
+Stukenborg
+Stuckemeyer
+Stsauveur
+Stroll
+Strohmeier
+Strissel
+Strimple
+Stremmel
+Streczywilk
+Strawhorn
+Stratz
+Stratos
+Straton
+Strassner
+Strama
+Strada
+Stoss
+Storti
+Stomberg
+Stolze
+Stoliker
+Stoler
+Stolberg
+Stolarik
+Stohlton
+Stofko
+Stofflet
+Stoff
+Stoesser
+Stoeber
+Stodden
+Stobierski
+Stobbs
+Stjohns
+Stirrup
+Stirman
+Stinehelfer
+Stimmell
+Stimits
+Stigger
+Stiers
+Stieff
+Stidam
+Stewarts
+Stevinson
+Stevey
+Sterett
+Ster
+Steppello
+Stepnoski
+Stentzel
+Stencil
+Stencel
+Stempien
+Steketee
+Steinbruckner
+Steinborn
+Steigman
+Steiber
+Stegent
+Steffani
+Steerman
+Steenken
+Steenhard
+Steedman
+Steckley
+Stealey
+Stayrook
+Stavnes
+Stauss
+Stash
+Stary
+Stare
+Stant
+Stanfa
+Standfield
+Standberry
+Standage
+Stanco
+Stanage
+Stampe
+Stamdifer
+Stalworth
+Stalma
+Staires
+Staines
+Staine
+Stahlberg
+Stadden
+Staberg
+Stabel
+Spurgers
+Spruce
+Sprinkel
+Springman
+Spriggle
+Sporleder
+Sporcic
+Spontak
+Sponholz
+Spohr
+Spittle
+Spiry
+Spiece
+Spicuzza
+Sperlich
+Sperdute
+Sperazza
+Spelts
+Speares
+Speakes
+Sparhawk
+Spaniel
+Spaar
+Soyars
+Soverns
+Southam
+Sour
+Souphom
+Soun
+Soula
+Sossamon
+Sosh
+Sosby
+Sorsby
+Soroka
+Soricelli
+Sorgi
+Sorbera
+Soplop
+Soohoo
+Sonoda
+Sonny
+Sonneborn
+Somodi
+Sommese
+Solman
+Sollie
+Solla
+Solina
+Soliani
+Soley
+Solecki
+Solages
+Sohre
+Soenksen
+Sodeman
+Sobiech
+Soberanis
+Snobeck
+Snerling
+Sneider
+Snaza
+Smolic
+Smigel
+Smigaj
+Smiechowski
+Smida
+Smerkar
+Smeby
+Slothower
+Slotemaker
+Slodysko
+Slivka
+Slimmer
+Slight
+Slifko
+Slayter
+Slawski
+Slauson
+Slatten
+Slain
+Skultety
+Skrip
+Skowyra
+Skorupa
+Skordahl
+Skomsky
+Skoff
+Sklenar
+Skeldon
+Skeesick
+Skea
+Skagen
+Sjostrand
+Sixtos
+Sivyer
+Siverson
+Siverling
+Sivan
+Siva
+Sitzler
+Sither
+Siskind
+Siske
+Siron
+Siregar
+Sirbaugh
+Sirak
+Siptak
+Sinstack
+Sins
+Siniscalchi
+Singlton
+Sinden
+Sinagra
+Sina
+Simpon
+Simmoneau
+Simler
+Simkulet
+Simi
+Simeona
+Simens
+Silverstone
+Silverness
+Silsbee
+Sillas
+Sileo
+Silbert
+Sikula
+Siglin
+Sigley
+Sigafus
+Siew
+Sietsma
+Sierras
+Siembida
+Sieker
+Siedlik
+Sidur
+Sidell
+Siddoway
+Sibille
+Sibilia
+Sibbald
+Shusta
+Shuskey
+Shurts
+Shryack
+Shroll
+Showell
+Shove
+Shoulars
+Shortino
+Shopp
+Shmidt
+Shiu
+Shirar
+Shinners
+Shingles
+Shinabery
+Shimko
+Shibles
+Shertzer
+Sherrin
+Sherril
+Shellhamer
+Shellhaas
+Sheldrup
+Sheladia
+Shehab
+Sheff
+Sheck
+Shearman
+Sheaff
+Shauer
+Shatswell
+Shaske
+Sharick
+Shappard
+Shallcross
+Shala
+Shaklee
+Shakespear
+Shafe
+Shady
+Shadwell
+Shacklett
+Seymor
+Settlemire
+Setting
+Sether
+Sesma
+Sesareo
+Seryak
+Serven
+Sers
+Serbus
+Serb
+Seppi
+Sephus
+Sentinella
+Sensel
+Senf
+Senato
+Sempek
+Semidey
+Semasko
+Selz
+Seltz
+Selmer
+Selitto
+Selim
+Seiser
+Seikel
+Seigle
+Seid
+Segouia
+Segner
+Segerson
+Segala
+Sefcik
+Seeholzer
+Seegert
+Sedita
+Sedenko
+Sedar
+Secondo
+Seckinger
+Sebald
+Seba
+Seahorn
+Seabright
+Scotty
+Scothorn
+Scordato
+Scoma
+Scobie
+Scipione
+Sciara
+Schwieterman
+Schwendemann
+Schwede
+Schwartzbach
+Schwarcz
+Schwalen
+Schutzman
+Schunemann
+Schulweis
+Schul
+Schuffert
+Schuckers
+Schrull
+Schrubbe
+Schreyer
+Schreckhise
+Schreader
+Schoonhoven
+Schoolman
+Schol
+Schoettmer
+Schoepf
+Schoenle
+Schoenecker
+Schobert
+Schnyer
+Schnoke
+Schnipper
+Schneiter
+Schneekloth
+Schnapp
+Schmits
+Schmelzle
+Schmelz
+Schmeisser
+Schmeiser
+Schmahl
+Schlotzhauer
+Schlott
+Schlossberg
+Schlipf
+Schlicker
+Schleuder
+Schleimer
+Schlauch
+Schlau
+Schlaefer
+Schiesser
+Schieler
+Schied
+Schie
+Scheuvront
+Scheumann
+Scherz
+Scheperle
+Schenewerk
+Schemm
+Schellenger
+Schaupp
+Schauf
+Schaudel
+Schau
+Schatzberg
+Scharr
+Schappert
+Schapp
+Schamel
+Schallhorn
+Schaefers
+Schadt
+Schadel
+Schackow
+Schabowski
+Schabes
+Schabert
+Schab
+Schaab
+Scavotto
+Scarver
+Scarsella
+Scarbro
+Scampoli
+Scammon
+Scallon
+Scalley
+Scale
+Scafuri
+Scadden
+Scacco
+Sawchuk
+Saviano
+Saverchenko
+Savelli
+Savarino
+Satsky
+Satoe
+Sarwinski
+Sartorio
+Sartorelli
+Sarria
+Saro
+Sarna
+Sarkin
+Sarisky
+Sario
+Sarazin
+Sara
+Sapia
+Santmyer
+Santmier
+Santillana
+Santanna
+Santacroce
+Sansouci
+Sannes
+Sanez
+Sandvig
+Sandino
+Sandella
+Sanburg
+Samy
+Sammer
+Samit
+Salvucci
+Salvey
+Salvatori
+Salvant
+Salvage
+Salts
+Salton
+Saltarelli
+Salt
+Salome
+Sallade
+Saletta
+Salehi
+Saleeby
+Salameh
+Salama
+Salaiz
+Salafia
+Sakry
+Sako
+Sakash
+Saitta
+Sahu
+Sahara
+Saguil
+Sagrera
+Saglimben
+Sagi
+Saggio
+Sagen
+Safranek
+Safko
+Saeli
+Sadar
+Sacre
+Saccardi
+Saborido
+Sabins
+Sabet
+Sabbah
+Saale
+Rynne
+Rynders
+Rylands
+Rykowski
+Ruzbasan
+Ruwe
+Rutiaga
+Ruthledge
+Rutecki
+Rusu
+Russler
+Rurup
+Ruozzo
+Ruot
+Runels
+Rumphol
+Rumpel
+Rumpca
+Rullo
+Ruisi
+Ruic
+Ruhle
+Ruffaner
+Rufer
+Ruetz
+Ruesink
+Ruehle
+Ruedy
+Ruden
+Rubulcaba
+Rua
+Roya
+Rowald
+Rovner
+Rouselle
+Roura
+Roulston
+Rougeaux
+Rotty
+Rothery
+Rotert
+Rossler
+Roskowinski
+Rosiak
+Rosh
+Rosenstock
+Roselius
+Roscigno
+Rosaro
+Rosada
+Roperto
+Ropers
+Rookwood
+Rongo
+Rondinelli
+Ronda
+Ronchetti
+Romrell
+Rollinger
+Rola
+Rokos
+Rohwer
+Rohrscheib
+Rohlf
+Rogal
+Rogacion
+Roeschley
+Roers
+Roemen
+Roelofs
+Roekle
+Roehrich
+Rodriguel
+Rodges
+Rodeen
+Roddey
+Roddam
+Rocquemore
+Rockers
+Roccia
+Robishaw
+Robida
+Robichau
+Robertshaw
+Roberton
+Roberta
+Roberg
+Rob
+Roary
+Rizzuti
+Rizal
+Riveros
+Rittenour
+Risper
+Rippin
+Ripp
+Riola
+Riogas
+Rinner
+Ringus
+Ringhand
+Rinehardt
+Rinderer
+Rigotti
+Righetti
+Riggi
+Riggans
+Rigazio
+Rigatti
+Rifenburg
+Rieu
+Riehm
+Riegler
+Riech
+Riebau
+Ridgel
+Ridens
+Ridener
+Riddel
+Rickner
+Richardt
+Ricciardone
+Rhynard
+Rhyan
+Rhoderick
+Rho
+Rheinschmidt
+Rezak
+Reusing
+Rettkowski
+Retterath
+Retta
+Reshid
+Reppe
+Repke
+Reos
+Reome
+Rensen
+Renschler
+Renova
+Renollet
+Renison
+Reninger
+Rengers
+Rengel
+Renart
+Rena
+Relihan
+Reisen
+Reiniger
+Reindel
+Reil
+Reier
+Reh
+Reggio
+Regener
+Reekers
+Reeger
+Redmann
+Reddinger
+Redcay
+Reckling
+Rebert
+Reategui
+Reagin
+Reagen
+Readnour
+Razzano
+Raynolds
+Rayer
+Raybould
+Rawdon
+Ravotta
+Ravo
+Ravitz
+Ravert
+Rathert
+Raterman
+Ratel
+Raque
+Rapko
+Ransone
+Ransburg
+Rangnow
+Randon
+Rancifer
+Ramotar
+Ramones
+Ramone
+Ramire
+Ramin
+Rameres
+Rakoski
+Rajala
+Raithel
+Rainie
+Rainge
+Rainbow
+Raigoza
+Rahming
+Ragazzo
+Radomski
+Radish
+Radilla
+Raden
+Radde
+Racano
+Rabine
+Rabil
+Rabell
+Rabasca
+Quiterio
+Quinzi
+Quink
+Quinci
+Quilliams
+Quiller
+Quider
+Quenneville
+Quelch
+Queeley
+Quear
+Quattro
+Quastad
+Quaglieri
+Pyscher
+Pust
+Purtle
+Purtill
+Purdin
+Puorto
+Punja
+Pullem
+Pulfer
+Puleio
+Pujia
+Puetz
+Puehler
+Puebla
+Ptomey
+Przewozman
+Prysock
+Pruter
+Prunier
+Pruess
+Prudom
+Pruchnik
+Proveaux
+Prophit
+Promise
+Procknow
+Proby
+Pro
+Prive
+Preziosi
+Preza
+Prem
+Preite
+Preisser
+Pregler
+Precella
+Prazma
+Prats
+Prator
+Prakash
+Prahm
+Prader
+Pozniak
+Poxon
+Powledge
+Pouge
+Pott
+Postlewaite
+Posthumus
+Posnick
+Posley
+Poskey
+Porro
+Poreda
+Poppema
+Popat
+Pondexter
+Ponciano
+Pompilio
+Pommer
+Polosky
+Pollom
+Pollo
+Pollica
+Pollaro
+Polizio
+Polek
+Polack
+Polacek
+Poirot
+Poertner
+Poduska
+Pockrus
+Pochintesta
+Pluym
+Pluhar
+Pluck
+Pliner
+Pliml
+Plese
+Pleasent
+Playle
+Plasky
+Plane
+Plack
+Pizani
+Pitz
+Pittari
+Pitruzzello
+Pistorius
+Pistilli
+Pisha
+Piselli
+Pisco
+Piros
+Pirone
+Pirolli
+Pirman
+Pirkl
+Pirie
+Pique
+Pintado
+Pinkey
+Pingrey
+Pinger
+Pinelo
+Pilsner
+Pilley
+Pilgreen
+Piles
+Pila
+Pignatello
+Pietig
+Pierrott
+Pierron
+Pierceall
+Pieratt
+Pienta
+Piekos
+Piechota
+Picquet
+Pickar
+Picerno
+Piceno
+Phyfiher
+Phorng
+Phearsdorf
+Pharmes
+Phariss
+Pfuhl
+Pfenning
+Pezzetti
+Pevy
+Petzoldt
+Pettrey
+Pettas
+Petta
+Petross
+Petrochello
+Petriello
+Petrelli
+Petch
+Pestoni
+Pestano
+Pesick
+Pesavento
+Perzanowski
+Perrien
+Perrenoud
+Perque
+Peroff
+Perlas
+Perkerson
+Perisho
+Perich
+Perfect
+Peregrino
+Peregoy
+Perch
+Pequeno
+Penza
+Pensis
+Penquite
+Peniston
+Penister
+Pendola
+Pendergraph
+Pelle
+Pelczar
+Pelch
+Pela
+Pehler
+Pegoda
+Peelle
+Peeling
+Pedroni
+Pedlar
+Pedder
+Pecoraino
+Peckman
+Pechal
+Pebsworth
+Peasnall
+Peasant
+Pead
+Peacemaker
+Paytes
+Paysen
+Payn
+Pavletic
+Pavlat
+Pavlas
+Pavese
+Paup
+Paulis
+Patrice
+Patocka
+Pat
+Pastorino
+Pascocello
+Parthemer
+Parreira
+Parido
+Paretti
+Pardun
+Parchment
+Papstein
+Papps
+Papetti
+Papakostas
+Pantoni
+Panik
+Panfilov
+Panfil
+Pana
+Pampusch
+Pamperin
+Palmitessa
+Palmero
+Pallett
+Palilla
+Palese
+Palesano
+Palange
+Pagenkopf
+Padon
+Padmanabhan
+Padinha
+Packen
+Pacitto
+Pacchiana
+Pabich
+Oza
+Oyabu
+Overdorf
+Ourada
+Otukolo
+Otterbine
+Ottalagano
+Oto
+Other
+Otano
+Osting
+Ostiguy
+Osterholt
+Osley
+Oscarson
+Osaile
+Ortz
+Ortolano
+Ortea
+Orte
+Ortaga
+Orszulak
+Orser
+Orihuela
+Orejel
+Ordorica
+Ording
+Ordal
+Orbin
+Oransky
+Oppel
+Onsgard
+Ondrick
+Olsin
+Ollmann
+Olives
+Olavarria
+Olano
+Olafson
+Okuno
+Okuniewski
+Okuhara
+Okrent
+Okoniewski
+Okeke
+Ohs
+Ohotnicky
+Ohno
+Ohlund
+Ohlendorf
+Ohaire
+Ogaz
+Ogando
+Offield
+Odiorne
+Oclair
+Ockenfels
+Ochocki
+Ocamb
+Ocallahan
+Obleton
+Oberly
+Oberhelman
+Oberbeck
+Nylin
+Nydick
+Nwachukwu
+Nutzmann
+Nuque
+Nunz
+Nulle
+Nuffer
+Notti
+Nothum
+Nothnagel
+Notah
+Nossett
+Nose
+Nosbisch
+Norrix
+Norlien
+Norkin
+Nordon
+Nordmeyer
+Norat
+Nooe
+Nokleby
+Nofziger
+Noens
+Nivison
+Niu
+Nittler
+Nissalke
+Nishikawa
+Ninness
+Nin
+Nimon
+Nifong
+Niewieroski
+Nietzer
+Niemela
+Nicolette
+Nicoletta
+Nico
+Nickolas
+Nickless
+Nicklaw
+Niccoli
+Nibbs
+Neyland
+Newmark
+Newey
+Newbauer
+Nevwirth
+Neverman
+Neuser
+Neumaier
+Neufville
+Netzley
+Netzel
+Nettle
+Neiswonger
+Neiswender
+Neilan
+Neidhardt
+Neesmith
+Nebgen
+Navia
+Nate
+Nasuti
+Nasso
+Nassimi
+Nashe
+Nases
+Naro
+Nardo
+Narasimhan
+Naqvi
+Nanka
+Naman
+Nahrstedt
+Nagura
+Nagarajan
+Nadile
+Nabours
+Nabers
+Mysinger
+Mynear
+Muzzarelli
+Muthig
+Mustian
+Muskus
+Muskelly
+Musi
+Mushtaq
+Musca
+Murzynski
+Murzyn
+Murrillo
+Murello
+Murdy
+Murakawa
+Munsinger
+Munnell
+Munks
+Munkberg
+Mundorf
+Mummey
+Mullick
+Mulkin
+Mulhollen
+Mulgrew
+Mulderig
+Mulac
+Muehl
+Muddiman
+Muckerman
+Muckenthaler
+Much
+Mucciolo
+Mruczek
+Mrazek
+Mowat
+Moure
+Mould
+Motts
+Mosure
+Mossor
+Mossberg
+Mosler
+Mosha
+Moscrip
+Moschetti
+Mosbarger
+Morua
+Morss
+Morron
+Morrall
+Moroni
+Morioka
+Moricca
+Morgensen
+Morganson
+Moreshead
+Morely
+Morch
+Moras
+Morar
+Moranville
+Moralas
+Morak
+Moradel
+Moothart
+Moonen
+Monzingo
+Montpetit
+Montjoy
+Monteagudo
+Monoz
+Mongrain
+Mongon
+Mondejar
+Monas
+Monachino
+Momplaisir
+Momin
+Moment
+Molpus
+Molony
+Molner
+Molleda
+Molinski
+Molinelli
+Molfetta
+Molenda
+Molchan
+Mohseni
+Mogg
+Moerke
+Moenius
+Moehlman
+Modugno
+Modi
+Modest
+Moder
+Moch
+Moat
+Miyamura
+Mittlestadt
+Mittelstedt
+Mittelman
+Mitschelen
+Mitro
+Mitchan
+Misty
+Missey
+Misenhimer
+Mirra
+Mirjah
+Mirante
+Miosek
+Minteer
+Minrod
+Minning
+Minney
+Minnema
+Minium
+Minihane
+Minicucci
+Minecci
+Minchey
+Milota
+Millson
+Milloway
+Millonzi
+Millier
+Milley
+Millam
+Milillo
+Milbrath
+Mikowski
+Mikola
+Mikler
+Mihelic
+Mihaila
+Miesen
+Mierzejewski
+Mickels
+Michienzi
+Michalke
+Miazga
+Mezydlo
+Mezick
+Meynard
+Meylor
+Mexicano
+Metsker
+Metrick
+Meter
+Mestad
+Meske
+Mertins
+Merta
+Mersinger
+Merschman
+Merna
+Merila
+Meridieth
+Mergen
+Merel
+Menzella
+Menze
+Mentnech
+Menson
+Mensick
+Mennig
+Mendillo
+Memos
+Melroy
+Melochick
+Mells
+Mellgren
+Meline
+Melich
+Melena
+Melchiori
+Melching
+Melahn
+Meisler
+Meinerding
+Meilleur
+Meidlinger
+Mehner
+Megrabyan
+Megee
+Meeuwsen
+Medlar
+Medick
+Medema
+Mechler
+Mechanic
+Meadowcroft
+Mcpike
+Mcpeake
+Mcnell
+Mcneary
+Mcmutry
+Mcmeekin
+Mcmannus
+Mcluen
+Mclouth
+Mclerran
+Mcleoud
+Mclagan
+Mckone
+Mckneely
+Mckissic
+Mckinnell
+Mckillips
+Mckibbon
+Mckenty
+Mckennan
+Mckeeman
+Mckasson
+Mcinturf
+Mcinerny
+Mchan
+Mcgurn
+Mcguirl
+Mcgue
+Mcgrain
+Mcgonnell
+Mcglumphy
+Mcglauflin
+Mcginity
+Mcgibboney
+Mcgeough
+Mcgauley
+Mcgarvie
+Mcfatter
+Mcentegart
+Mcenroe
+Mcelmury
+Mcelhinny
+Mcdonnel
+Mcdoniel
+Mcdoe
+Mcdermond
+Mcdearmon
+Mcdearman
+Mcday
+Mcdannald
+Mcdaid
+Mccurren
+Mccrosky
+Mccrane
+Mccraig
+Mccooey
+Mccoo
+Mccolpin
+Mccolloch
+Mcclucas
+Mcclester
+Mcclement
+Mcclamroch
+Mcclammy
+Mcclallen
+Mccarte
+Mccaie
+Mccaddon
+Mcanelly
+Mcalmond
+Mcalary
+Mazzini
+Mazzarino
+Mazzara
+Mazzanti
+Mazurk
+Mazor
+Mayerle
+Mayenschein
+Mayard
+Mayans
+Maxedon
+Mavromatis
+Mavins
+Maves
+Mausser
+Maulsby
+Matya
+Matuke
+Matto
+Mattler
+Mattiace
+Matkowski
+Mathern
+Matero
+Matchette
+Matayoshi
+Matar
+Mastine
+Massing
+Massimo
+Masseria
+Massenberg
+Massard
+Masoud
+Masotti
+Maslak
+Masey
+Masella
+Mascarena
+Mascall
+Marzella
+Maryott
+Marwick
+Marugg
+Martt
+Martinis
+Martian
+Martha
+Marstaller
+Marsingill
+Marsicek
+Marotto
+Market
+Markegard
+Marke
+Marinella
+Marien
+Margison
+Margheim
+Margason
+Margaris
+Margaret
+Marett
+Marentes
+Marcott
+Marcon
+Marchena
+Marcellino
+Mapston
+Mantione
+Mantanona
+Mansouri
+Manoi
+Mankus
+Mankins
+Manin
+Manikas
+Mangieri
+Manfredini
+Mane
+Mandt
+Mandolini
+Mandley
+Mancina
+Manas
+Maltsberger
+Maltais
+Malmin
+Mallis
+Mallicoat
+Malleck
+Mallach
+Malkowski
+Malkani
+Malito
+Malensek
+Malandra
+Malander
+Makos
+Makanani
+Maille
+Mail
+Maidens
+Maid
+Mahowald
+Mahala
+Mahajan
+Magnotta
+Maggiore
+Magel
+Maestos
+Maerz
+Maedche
+Madise
+Madi
+Mades
+Maddaloni
+Madayag
+Madaras
+Macnair
+Mackinlay
+Mackesy
+Machon
+Machia
+Machey
+Machesky
+Machacek
+Maceyak
+Macchio
+Macbride
+Mabray
+Maasch
+Lyseski
+Lykken
+Luzania
+Luxenberg
+Lutrell
+Lupkes
+Lupino
+Lupardus
+Lunnon
+Lunghofer
+Lundvall
+Lundby
+Lundborg
+Lulow
+Lukman
+Lukin
+Lukaszewski
+Lukacs
+Lugones
+Luger
+Lueder
+Ludeke
+Lucek
+Lucchetti
+Lucchese
+Lozowski
+Lozaro
+Loyer
+Lowthert
+Lowdermilk
+Lovitz
+Lovinggood
+Lovenduski
+Loura
+Loung
+Lounder
+Louks
+Loughry
+Loudermill
+Lotta
+Lostetter
+Loskot
+Losiewski
+Lorman
+Loren
+Lorelli
+Lorange
+Lonsinger
+Longinotti
+Longhurst
+Lomedico
+Lola
+Lohwasser
+Lohn
+Lohden
+Lograsso
+Logie
+Loftman
+Loften
+Lofaso
+Loewer
+Loehrs
+Locy
+Loconte
+Lockerman
+Lockerby
+Locken
+Lobaton
+Loatman
+Lleras
+Lizak
+Livingood
+Litwiler
+Litvin
+Littledave
+Lites
+Lisee
+Lipszyc
+Lippy
+Lionello
+Linsday
+Linnear
+Linklater
+Lingbeck
+Lindie
+Lindenfelser
+Lindenberger
+Linarez
+Limber
+Lily
+Lightning
+Liffick
+Lieto
+Liestman
+Liepins
+Lieng
+Liebross
+Licciardi
+Licavoli
+Libbee
+Lhuillier
+Lhommedieu
+Leyra
+Lewman
+Levreault
+Levitre
+Levings
+Levick
+Levecke
+Levanger
+Leval
+Leva
+Leuthold
+Leuenthal
+Letze
+Letterlough
+Leski
+Lerwill
+Lertora
+Leppla
+Leopoldo
+Leonides
+Leonardis
+Lenoue
+Lenoch
+Lengerich
+Lemont
+Lemmert
+Lemery
+Lemaitre
+Lella
+Leko
+Leithauser
+Leisher
+Leise
+Leisch
+Leiendecker
+Leiber
+Leialoha
+Lehtomaki
+Lehigh
+Leggs
+Legate
+Leflar
+Lefeber
+Leezer
+Ledden
+Lecleir
+Lechliter
+Lebrane
+Lebarron
+Leason
+Leapheart
+Leadman
+Lazarte
+Lawin
+Lavole
+Lavesque
+Laverdure
+Lautner
+Lauthern
+Laurila
+Laurendeau
+Launderville
+Laumeyer
+Latina
+Laszlo
+Lassan
+Larzelere
+Larzazs
+Larubbio
+Larriuz
+Larew
+Laremont
+Laredo
+Lardizabal
+Larance
+Lappa
+Lapolla
+Lapatra
+Lapaglia
+Lantieri
+Lannan
+Lann
+Langwith
+Langolf
+Langloss
+Langlo
+Langholz
+Langhart
+Langfitt
+Langendorf
+Langenbach
+Langbehn
+Lanehart
+Landoni
+Landherr
+Landberg
+Landazuri
+Lancey
+Lamus
+Lamunyon
+Lampitt
+Lampiasi
+Lammon
+Lamme
+Lamirand
+Lambes
+Lamarta
+Lamarra
+Lalim
+Lalande
+Laky
+Laitila
+Laidler
+Laich
+Lahue
+Lahtinen
+Lagrasse
+Lagrand
+Lagle
+Lagerstrom
+Lagerberg
+Laferney
+Lacson
+Lachenauer
+Lablue
+Labean
+Lab
+Kuzara
+Kuza
+Kuy
+Kutchera
+Kustra
+Kurtyka
+Kurschner
+Kurka
+Kunstlinger
+Kunka
+Kunicki
+Kunda
+Kulling
+Kulla
+Kulbida
+Kuker
+Kujath
+Kujala
+Kuhta
+Kuhner
+Kuhle
+Kufalk
+Kuennen
+Kuen
+Kudley
+Kucharik
+Kuca
+Kubic
+Kryst
+Krysh
+Krumenauer
+Kruczek
+Kroschel
+Kronk
+Kroells
+Krivak
+Kristoff
+Kristin
+Kreuziger
+Kreitz
+Kreisberg
+Kreiman
+Kreighbaum
+Kreh
+Kreck
+Kraszewski
+Krason
+Krammes
+Krake
+Kozusko
+Kozola
+Kozikowski
+Kozielski
+Kowis
+Kowalske
+Kottman
+Kottler
+Kottenstette
+Kostelnick
+Kosmowski
+Koska
+Kosinar
+Kosik
+Kosanovic
+Kosanke
+Kortge
+Korsak
+Kornbau
+Kordas
+Korby
+Korbel
+Kopperman
+Koppenhaver
+Kopischke
+Koper
+Kopelman
+Kopel
+Kopas
+Kooser
+Koors
+Koor
+Koone
+Koogle
+Konzen
+Konieczka
+Kondracki
+Kondos
+Komatsu
+Kolo
+Kolarik
+Kolacki
+Kokesh
+Kohrt
+Kohrs
+Kogel
+Kofron
+Kofman
+Koewler
+Koetting
+Koes
+Koellner
+Koellmann
+Koczela
+Kocon
+Knoth
+Knollman
+Knoebel
+Knknown
+Knittle
+Kniphfer
+Knightly
+Kniffin
+Knaphus
+Knaak
+Kloth
+Klonoski
+Kloke
+Kloer
+Klinetob
+Kliger
+Klich
+Kleyman
+Klepchick
+Klemish
+Kleen
+Klebe
+Klakowicz
+Klaft
+Kithcart
+Kister
+Kisker
+Kishel
+Kishbaugh
+Kirt
+Kirouac
+Kirley
+Kirklen
+Kirkegaard
+Kirchen
+Kipka
+Kipfer
+Kinsinger
+Kiniry
+Kinikini
+Kingma
+Kinderknecht
+Kinahan
+Kimmes
+Kimak
+Killiany
+Killelea
+Kilkus
+Kilfoyle
+Kiflezghie
+Kiffer
+Kiesewetter
+Kienow
+Kieler
+Kiebler
+Kicks
+Kicker
+Kibel
+Kibe
+Kibbee
+Kiang
+Khounthavong
+Khatri
+Khamsyuorauon
+Kham
+Keye
+Keup
+Keto
+Ketch
+Kess
+Kerth
+Kero
+Kernell
+Kerkvliet
+Keomany
+Keomanivong
+Kennemur
+Kennel
+Kenndey
+Kendi
+Kempter
+Kempinski
+Kemna
+Kellan
+Keliikoa
+Keledjian
+Keithan
+Keisel
+Keib
+Kehs
+Kedley
+Keay
+Kearin
+Kawulok
+Kawai
+Kawaa
+Kava
+Kaunisto
+Kaumo
+Kauahi
+Kattner
+Katra
+Kastel
+Kastein
+Kassulke
+Kassman
+Kassing
+Kashani
+Kasch
+Karty
+Karstetter
+Karrenberg
+Karper
+Karow
+Karmo
+Karhoff
+Kardell
+Kardas
+Karapetian
+Kapper
+Kappen
+Kapichok
+Kanis
+Kaneakua
+Kanaris
+Kamuda
+Kamirez
+Kamat
+Kaloudis
+Kallberg
+Kallaher
+Kalkwarf
+Kalkman
+Kalk
+Kalisek
+Kalehuawehe
+Kalchik
+Kalbfleisch
+Kalberer
+Kalal
+Kala
+Kakimoto
+Kaing
+Kaigle
+Kahill
+Kahanaoi
+Kaemmerling
+Kadri
+Kadle
+Kading
+Kadi
+Kadar
+Kachmar
+Kachiroubas
+Kachelmeyer
+Kaase
+Juve
+Juul
+Justinger
+Jungwirth
+Jungman
+Jungck
+Julander
+Juenemann
+Jubie
+Joun
+Joswick
+Jossund
+Joss
+Jory
+Jonnson
+Jongsma
+Joliet
+Johngrass
+Jocoy
+Jing
+Jimerez
+Jimbo
+Jeudy
+Jerowski
+Jernstrom
+Jernstad
+Jernberg
+Jeoffroy
+Jentry
+Jennie
+Jeng
+Jenaye
+Jemerson
+Jeltema
+Jeanpaul
+Jeanmard
+Jax
+Javery
+Jaudon
+Jasperse
+Jasmer
+Jarred
+Jarrar
+Jargas
+Jardot
+Jardell
+Jaquay
+Jappa
+Janower
+Jankoski
+Janise
+Jandrey
+Jandl
+Jakubiak
+Jakobson
+Jakobsen
+Jahncke
+Jagers
+Jacobitz
+Jackon
+Izard
+Ivel
+Itzkowitz
+Itani
+Issacs
+Isome
+Isle
+Islar
+Isidro
+Isidoro
+Isch
+Irvan
+Irizary
+Irene
+Ipson
+Ip
+Ioele
+Interiano
+Insalaco
+Iniestra
+Ingargiola
+Impson
+Illiano
+Iller
+Illa
+Ilardi
+Iida
+Ihrke
+Igneri
+Igbal
+Igartua
+Iffland
+Idell
+Iberra
+Iba
+Ianacone
+Hysong
+Hyrkas
+Huzzard
+Huttle
+Husselbee
+Husseini
+Hupe
+Hunzeker
+Hunnicut
+Humprey
+Humbird
+Humason
+Hugle
+Hufana
+Huestis
+Huesing
+Huell
+Hudy
+Hudley
+Hudas
+Hudalla
+Hudack
+Huckfeldt
+Hubka
+Hubenthal
+Huante
+Hsing
+Hromek
+Hritz
+Hrdlicka
+Howzell
+Howles
+Howat
+Hovarter
+Houy
+Housler
+Houska
+Houseal
+Houlberg
+Hostert
+Hosman
+Hoscheid
+Horvers
+Hortin
+Hornish
+Hornbeak
+Hornaday
+Hoppman
+Hopfer
+Hoot
+Honts
+Honsberger
+Hons
+Honnen
+Honberger
+Honahnie
+Homma
+Homesley
+Holyoak
+Holweger
+Holubar
+Holtzer
+Holtrop
+Holtberg
+Holpp
+Holmquest
+Hollinghead
+Holje
+Holgerson
+Holabaugh
+Hoitt
+Hofford
+Hoffmaster
+Hoffine
+Hoffelt
+Hoes
+Hoellwarth
+Hoegh
+Hoegerl
+Hoeger
+Hodrick
+Hodgkiss
+Hodek
+Hockey
+Hobday
+Hlavacek
+Hlad
+Hitzeman
+Hitzel
+Hitsman
+Hissong
+Hissam
+Hiscock
+Hirz
+Hirshberg
+Hipkins
+Hinsch
+Hinken
+Hinckle
+Hinchliff
+Himmons
+Himmelwright
+Himmelspach
+Himebaugh
+Hilst
+Hilmes
+Hillsgrove
+Hillestad
+Hillesland
+Hillegass
+Hilfiger
+Hilado
+Highshaw
+Highers
+Higginbothan
+Higbie
+Hieronymus
+Hidy
+Hickory
+Hickernell
+Hibma
+Hibbets
+Heximer
+Hewgley
+Heutmaker
+Heuschkel
+Heupel
+Heumann
+Heuman
+Hetzer
+Hetherman
+Hesterman
+Hespe
+Hertweck
+Herson
+Herry
+Herrboldt
+Herms
+Hermosilla
+Herl
+Herbolsheimer
+Herbel
+Hera
+Heptinstall
+Heppler
+Heppell
+Henslin
+Henschen
+Hennington
+Hennagir
+Henkhaus
+Henken
+Henggeler
+Hempfling
+Hemmerling
+Hemish
+Hema
+Helveston
+Helsey
+Helscher
+Helo
+Heline
+Helfin
+Helder
+Heitner
+Heiple
+Heinzelman
+Heinricher
+Heines
+Heimsness
+Heiler
+Heidelburg
+Heiberg
+Hegner
+Hegler
+Hefferman
+Heffelbower
+Heebner
+Hediger
+Hedding
+Heckbert
+Hearnsberger
+Heaivilin
+Heagle
+Heafner
+Hazelrig
+Hayth
+Hayoz
+Haydu
+Haybarger
+Haya
+Havers
+Haverfield
+Hauze
+Haugabrook
+Haub
+Hathcoat
+Hasychak
+Hassin
+Hassey
+Hasenberg
+Hasek
+Harvat
+Haruta
+Hartvigsen
+Hartong
+Hartke
+Harre
+Harradon
+Harnisch
+Harmond
+Harmening
+Harlem
+Harkrader
+Harklerode
+Hargitt
+Hardon
+Hardgrave
+Hardester
+Harbeson
+Harben
+Hanrath
+Handville
+Handcock
+Hamza
+Hamson
+Hamming
+Hamic
+Hambley
+Halphen
+Halpain
+Halmes
+Hallaway
+Hallauer
+Half
+Haldiman
+Halbur
+Hakkila
+Hakimian
+Haimes
+Hahs
+Hagmann
+Hagglund
+Hagert
+Hagee
+Hafeman
+Haeber
+Haddan
+Hada
+Hackner
+Hackel
+Hacher
+Habisch
+Haarstad
+Haare
+Haaker
+Gyger
+Guzowski
+Guzi
+Guzalak
+Guyon
+Guyll
+Gutzmer
+Guttirez
+Gutt
+Gutierrex
+Gutierre
+Gut
+Gustis
+Gushwa
+Gurke
+Gurevich
+Gunyan
+Gumz
+Guisbert
+Guire
+Guintanilla
+Guimaraes
+Guillereault
+Guidos
+Guidera
+Guffin
+Guererro
+Guenthner
+Guedes
+Guareno
+Guardian
+Grussing
+Gruska
+Grudzien
+Growcock
+Grossenbacher
+Grosjean
+Groshans
+Grondahl
+Grollimund
+Groeneveld
+Groenendyk
+Grinnan
+Grindell
+Grindeland
+Grimaud
+Grigorov
+Griffard
+Grierson
+Grich
+Gribbins
+Gribbin
+Grever
+Gretter
+Grennon
+Grenfell
+Gremer
+Greising
+Greenhoward
+Gravitz
+Gravis
+Gravino
+Graubard
+Grates
+Granstrom
+Grannell
+Grandt
+Granat
+Grambling
+Gramajo
+Gralak
+Graise
+Grafe
+Grade
+Grad
+Gracy
+Goyco
+Goyal
+Govindeisami
+Govert
+Govero
+Gouras
+Goulbourne
+Goularte
+Gouker
+Gotwalt
+Gottshall
+Gottsch
+Gorum
+Gordo
+Gordils
+Gorbet
+Goonan
+Goombi
+Gooley
+Goolesby
+Goodlet
+Goodland
+Gomaz
+Golt
+Golombek
+Golom
+Golojuch
+Golightley
+Goldyn
+Goldkamp
+Goldfine
+Goldermann
+Goffinet
+Goetter
+Goethals
+Goerdt
+Goehl
+Goedken
+Goede
+Goedde
+Goeckel
+Godshall
+Godleski
+Godino
+Godine
+Godden
+Godar
+Gockley
+Gockel
+Gochnour
+Gobler
+Goard
+Gniewek
+Gnerre
+Gluszek
+Glunt
+Glotzbach
+Glory
+Glista
+Glisan
+Glende
+Glee
+Gleave
+Glaus
+Glau
+Glassing
+Gladhill
+Gizzo
+Giulian
+Gittins
+Girven
+Girt
+Girling
+Girardot
+Gipp
+Giovannini
+Gionet
+Gins
+Ginolfi
+Gimar
+Gilvin
+Gilliom
+Gilling
+Gillece
+Gilio
+Gildow
+Gilberg
+Gieser
+Gierisch
+Gielow
+Gieck
+Gica
+Gibboney
+Giarraputo
+Gianopoulos
+Giannecchini
+Giambruno
+Ghrist
+Ghiloni
+Geving
+Getto
+Gessford
+Gesner
+Gesick
+Gerstenkorn
+Gersbach
+Geroge
+Gerleman
+Gerl
+Gerkin
+Gerding
+Gerchak
+Georgiades
+Geoffroy
+Gentes
+Genre
+Genous
+Genge
+Geney
+Gendusa
+Gendel
+Gemma
+Gembler
+Gemaehlich
+Geldmacher
+Gehris
+Geffrard
+Geffken
+Geans
+Gavel
+Gavaldon
+Gaughran
+Gaud
+Gaucin
+Gauch
+Gattuso
+Gatliff
+Gather
+Gastonguay
+Gassen
+Gasior
+Garzia
+Gartz
+Gartley
+Garski
+Garramone
+Garoner
+Garone
+Garnow
+Garley
+Garibai
+Garguilo
+Garfunkel
+Gardley
+Gardecki
+Garcilazo
+Garbarini
+Garan
+Garafalo
+Gani
+Gandert
+Gampong
+Gamons
+Gamma
+Gambone
+Gambler
+Galves
+Galo
+Galm
+Galluccio
+Gallinari
+Gallentine
+Gallamore
+Galeotti
+Galella
+Gajica
+Gaisford
+Gaietto
+Gahlman
+Gahl
+Gaglia
+Gaffke
+Gaetz
+Gadwah
+Gabaree
+Gaar
+Fust
+Furutani
+Furner
+Furnace
+Furgison
+Furgeson
+Fundis
+Fullem
+Fullagar
+Fujisawa
+Fugit
+Fugh
+Fuemmeler
+Fuelling
+Fude
+Frusci
+Frosch
+Frontera
+Fronek
+Fritzman
+Fristoe
+Frishkorn
+Frilling
+Frigge
+Friels
+Friehe
+Friedline
+Fridlington
+Frezzo
+Frezza
+Fresta
+Freise
+Freiman
+Freidhof
+Freiberger
+Freetage
+Freet
+Freemyer
+Fredin
+Fredenberg
+Frayne
+Fraughton
+Franzel
+Frankie
+Frankenstein
+Frankenberg
+Francher
+Franch
+Francesconi
+Franc
+Fraize
+Fragmin
+Frabott
+Foxman
+Fouty
+Fournet
+Foulcard
+Fouhy
+Fougere
+Fotopoulos
+Forsmark
+Fornell
+Form
+Forline
+Forguson
+Fontus
+Fontanella
+Folkner
+Fok
+Foggie
+Fogelman
+Flumerfelt
+Fluegge
+Fluegel
+Fluck
+Floe
+Flocco
+Flitsch
+Flirt
+Flinders
+Fletchen
+Flechsig
+Flebbe
+Flathers
+Flatau
+Flamer
+Flaharty
+Fladger
+Fitten
+Fitchpatrick
+Fissori
+Fissel
+Fischler
+Fioritto
+Fiori
+Fiorentini
+Fiorella
+Finnemore
+Finkelson
+Fingleton
+Fingerhut
+Finazzo
+Filmer
+Fillip
+Fillingham
+Filipek
+Filan
+Figurski
+Figueron
+Figueiras
+Figley
+Fiedor
+Ficker
+Fickas
+Fevig
+Feutz
+Fetner
+Fertal
+Ferraiolo
+Fernsler
+Fernet
+Fernatt
+Fergusen
+Ferg
+Feraco
+Fenny
+Fengler
+Felsted
+Fellner
+Fellin
+Fellenz
+Felkner
+Felkel
+Feliu
+Feleppa
+Felderman
+Felde
+Feigel
+Feickert
+Feibusch
+Fedorek
+Fedora
+Federgreen
+Fedalen
+Feck
+Febre
+Fearnow
+Feagler
+Favorito
+Faville
+Favalora
+Fauls
+Faudree
+Fasulo
+Fassino
+Farson
+Farlin
+Faretra
+Farenbaugh
+Farella
+Faraone
+Faragoza
+Fanucchi
+Fantroy
+Fanny
+Fangman
+Famiglietti
+Faltus
+Faltin
+Falt
+Falley
+Falldorf
+Falick
+Fala
+Fahrney
+Faggs
+Fafard
+Faes
+Fadely
+Fadel
+Facchine
+Fabionar
+Ezagui
+Evoy
+Evilsizer
+Evick
+Eversoll
+Eversman
+Everley
+Evelo
+Euvrard
+Eun
+Etkin
+Ethen
+Estrela
+Esteb
+Estain
+Estacion
+Esquerra
+Esposto
+Espert
+Eskra
+Eskin
+Eskenazi
+Eshom
+Eshenbrenner
+Esera
+Escobio
+Eschief
+Eschenbrenner
+Erschen
+Erlewine
+Erdner
+Erck
+Erceg
+Erbach
+Epolito
+Ephriam
+Enwright
+Enwall
+Entrikin
+Entress
+Entler
+Enstad
+Engwall
+Engroff
+Englemann
+Engelson
+Enderlin
+Enamorado
+Emme
+Emlay
+Emke
+Emerton
+Embertson
+Elworthy
+Elwick
+Elward
+Eloy
+Ellyson
+Ellstrom
+Ellingboe
+Elliam
+Elifritz
+Elgart
+Elerick
+Eitzen
+Eismann
+Eisentrout
+Eischeid
+Eirich
+Eikner
+Eickhorst
+Ehrler
+Ehrle
+Eglinton
+Egerer
+Egelhoff
+Edmunson
+Ecord
+Eckrich
+Eckland
+Echevaria
+Ebersold
+Eberenz
+Ebener
+Ebadi
+Ealand
+Eaks
+Eagleston
+Eaglen
+Eagin
+Dyals
+Dwelley
+Duy
+Duva
+Dutter
+Dutko
+Duster
+Duskin
+Dusel
+Durrenberger
+Durke
+Durian
+Dupay
+Duntley
+Dunsford
+Dundee
+Dulemba
+Dugi
+Dufficy
+Duensing
+Dueno
+Dueitt
+Duclo
+Dubrock
+Dubitsky
+Drumgo
+Drozdowicz
+Dromgoole
+Drobot
+Drivas
+Drinkwine
+Drewing
+Dressman
+Dreessen
+Drainville
+Dragna
+Draffin
+Dowgiallo
+Dovey
+Dougher
+Dottin
+Dossous
+Dossie
+Dose
+Doronio
+Dorning
+Dorko
+Dorion
+Dorinirl
+Doring
+Doorn
+Donohoo
+Donnally
+Donkin
+Donez
+Donerson
+Dondlinger
+Donchez
+Donaway
+Donatien
+Donath
+Dommel
+Domine
+Domin
+Domiano
+Domhoff
+Domek
+Doller
+Dolinsky
+Dolberry
+Doker
+Doil
+Doidge
+Dohman
+Doeden
+Dodridge
+Dodgson
+Dobkowski
+Dobie
+Dobes
+Dobert
+Diwan
+Ditomasso
+Distaffen
+Distad
+Dispenza
+Disorbo
+Diskind
+Diserens
+Discipio
+Dirico
+Dire
+Dirago
+Diprima
+Dinwoodie
+Dinn
+Dinkens
+Dinius
+Dingeldein
+Dimon
+Dimitt
+Dimitriadis
+Dilliard
+Dilick
+Dilauro
+Dilallo
+Dilalla
+Dihel
+Digilio
+Difonzo
+Difeo
+Dietze
+Dietl
+Diesi
+Diesel
+Dieppa
+Dienes
+Diemert
+Diegel
+Dieffenbacher
+Diec
+Dickhoff
+Dickensheets
+Dibonaventura
+Dibblee
+Dibartolo
+Dibacco
+Dhondt
+Dewer
+Develbiss
+Devazier
+Devara
+Deuser
+Deur
+Deuell
+Detzel
+Dettling
+Detro
+Destine
+Destefanis
+Desorcy
+Desomma
+Deslandes
+Desisto
+Desiga
+Deshler
+Deshaw
+Desgroseillie
+Desaulniers
+Derwitsch
+Derrig
+Derouchie
+Dermady
+Derider
+Derfus
+Derbes
+Depperschmidt
+Depoyster
+Depaula
+Dense
+Dennin
+Deniro
+Denio
+Dengel
+Deneen
+Dempsy
+Demmy
+Demmert
+Demichelis
+Demedeiros
+Dembroski
+Dembitzer
+Demarse
+Demaranville
+Demagistris
+Deluz
+Delson
+Delrossi
+Delrie
+Delossanto
+Delos
+Delmolino
+Dellis
+Dellarocco
+Dellano
+Della
+Delisser
+Delille
+Deleston
+Delerme
+Deleone
+Delehanty
+Delbalso
+Delavina
+Delauter
+Delashmit
+Dekalb
+Deguire
+Degross
+Degroote
+Degrasse
+Degrange
+Degrace
+Degasperis
+Deffibaugh
+Defaber
+Decrosta
+Decristoforo
+Dechert
+Decelle
+Decapua
+Decapite
+Decandia
+Debuse
+Debruler
+Deblauw
+Debella
+Debeer
+Dayrit
+Davidian
+Davick
+Davich
+Davia
+Daversa
+Davern
+Davault
+Dautrich
+Dausch
+Dathe
+Dastrup
+Dassow
+Darras
+Darnold
+Darks
+Dargis
+Dargatz
+Darbouze
+Dannenfelser
+Dannard
+Dampf
+Dalzen
+Dalphonse
+Dalluge
+Dalhover
+Daivs
+Dainack
+Daher
+Dagle
+Daghita
+Dagdag
+Dafonseca
+Daffern
+Daehler
+Dadson
+Czuba
+Czlapinski
+Czarnik
+Czap
+Cynova
+Cwiklinski
+Cuzco
+Cutno
+Curt
+Curbow
+Cunninghan
+Cunis
+Cuningham
+Cunico
+Culmer
+Cuhel
+Cuestas
+Cuebas
+Cuchares
+Cubr
+Csizmadia
+Crumpacker
+Cruell
+Crousore
+Crosten
+Crosman
+Crooked
+Cromuel
+Cromey
+Crockarell
+Croan
+Crissler
+Crispen
+Crismon
+Crise
+Criscillis
+Crippin
+Crilly
+Cresta
+Cregar
+Cragun
+Coye
+Cowing
+Cower
+Coverstone
+Coverdell
+Couty
+Coutant
+Courtnage
+Courteau
+Couper
+Countee
+Coultas
+Coughran
+Cottew
+Cotler
+Cotelesse
+Costen
+Cossin
+Coskrey
+Cosen
+Cosden
+Corvera
+Cortis
+Corsello
+Corrion
+Corrigeux
+Correiro
+Coro
+Cornetta
+Corneil
+Corlee
+Corin
+Corgan
+Corfman
+Corell
+Cordovi
+Cordia
+Cordas
+Corcino
+Corchero
+Coral
+Coppolino
+Coppernoll
+Coppens
+Coote
+Cooperstein
+Cooperrider
+Conterras
+Consolazio
+Cons
+Connin
+Connerley
+Conkin
+Congress
+Concienne
+Conaghan
+Comrey
+Cominsky
+Comella
+Comee
+Come
+Combe
+Coln
+Collums
+Collamore
+Colicchio
+Colee
+Colding
+Colder
+Colbenson
+Colagiovanni
+Cokely
+Coin
+Codde
+Cobrin
+Coak
+Cluxton
+Cluesman
+Clouston
+Closser
+Clopp
+Cliatt
+Clendennen
+Clearman
+Clattenburg
+Clarks
+Clapsaddle
+Cius
+Cira
+Ciolli
+Cinotti
+Cimko
+Cima
+Cienega
+Cicatello
+Cicale
+Ciarlante
+Cianfrini
+Cianciulli
+Churley
+Churches
+Chuong
+Chukes
+Christou
+Christescu
+Christe
+Chrismon
+Chrisler
+Choun
+Chobot
+Chisem
+Chiong
+Chimera
+Chila
+Chicca
+Chiarito
+Chhun
+Chhum
+Chhim
+Chestang
+Chesler
+Cherubin
+Chernosky
+Cherebin
+Chepiga
+Chellis
+Chell
+Cheda
+Checca
+Cheater
+Cheatem
+Chaulk
+Chaudhuri
+Chauca
+Chatcho
+Chartraw
+Charping
+Charnley
+Charm
+Charlson
+Charbonneaux
+Charan
+Chapp
+Chango
+Chanez
+Chancer
+Chamnanphony
+Chalepah
+Chaiken
+Chaddlesone
+Chaconas
+Chabaud
+Cestia
+Cessor
+Cervetti
+Cerveny
+Cerise
+Cerecer
+Cerasoli
+Cera
+Centini
+Cenci
+Cembura
+Celli
+Cederstrom
+Cdebaca
+Cayo
+Cawthron
+Caviggia
+Cavers
+Caveney
+Causley
+Caughlin
+Cathie
+Catan
+Catala
+Castrogiovann
+Castleton
+Castilo
+Castillio
+Castellaw
+Castellari
+Castejon
+Caspersen
+Casivant
+Cashio
+Cascioli
+Casciano
+Casamento
+Casadei
+Carwin
+Carvin
+Carucci
+Cartin
+Cartez
+Carston
+Carrio
+Carriaga
+Carretino
+Carotenuto
+Carosiello
+Carolfi
+Carnathan
+Carnalla
+Carnagey
+Carlill
+Carinio
+Cariker
+Caride
+Care
+Cardero
+Cardenal
+Carasquillo
+Carabez
+Capwell
+Capurro
+Capulong
+Cappucci
+Cappetta
+Cappa
+Capouch
+Caporali
+Caponigro
+Capilla
+Capata
+Capan
+Canzoneri
+Cantine
+Cantarano
+Cannellos
+Cannard
+Cannada
+Canlas
+Cangey
+Canaan
+Campoy
+Campany
+Campainha
+Cambi
+Camba
+Camastro
+Camano
+Calrk
+Callin
+Callari
+Calicutt
+Calemine
+Caleb
+Caldon
+Caldas
+Cajas
+Cadelina
+Cacal
+Cabriales
+Cables
+Bytheway
+Byland
+Byes
+Byan
+Buzick
+Buziak
+Buzhardt
+Butzlaff
+Buttolph
+Butta
+Butron
+Butorac
+Butaud
+Butac
+Busuttil
+Busque
+Busing
+Busboom
+Burwood
+Burright
+Burri
+Burrall
+Burness
+Burlington
+Burlin
+Burkham
+Burick
+Burich
+Burgner
+Burdex
+Burdell
+Burde
+Burba
+Buol
+Bundi
+Bulick
+Bulgin
+Bukovsky
+Bukovac
+Bujak
+Bugett
+Buffo
+Bueschel
+Bueckers
+Budnik
+Buckey
+Buckel
+Buchko
+Buchinski
+Buchana
+Buchaman
+Bucek
+Buba
+Bryans
+Brustkern
+Brussel
+Brusseau
+Bruntz
+Brunscheen
+Brunken
+Brumbach
+Bruess
+Brueckman
+Brueck
+Brucken
+Brozena
+Brozek
+Brownley
+Browers
+Brosman
+Brosch
+Broody
+Brood
+Bronzo
+Bronn
+Bromwell
+Brome
+Bromagen
+Broll
+Brofman
+Broekemeier
+Brodi
+Brixner
+Brisban
+Brinkmeier
+Bringham
+Bridgforth
+Bridgette
+Breznak
+Brewbaker
+Breitweiser
+Breiten
+Breitbarth
+Brehaut
+Breedan
+Breech
+Bree
+Bredernitz
+Brechner
+Brechbiel
+Breashears
+Brazinski
+Brazille
+Bratz
+Bratu
+Bratsch
+Bras
+Branting
+Brannin
+Bramsen
+Brailford
+Bragas
+Bradney
+Bradner
+Bradigan
+Bradica
+Brad
+Brabston
+Bozwell
+Boys
+Boyn
+Boyar
+Boyance
+Boxton
+Bowering
+Bowar
+Bournazian
+Bourgue
+Bourgoine
+Bourdage
+Boulier
+Boulds
+Boulding
+Bouch
+Bottum
+Bottorf
+Botero
+Bossler
+Bosshardt
+Bossart
+Bosman
+Borzillo
+Borstad
+Borsos
+Borsellino
+Borrayo
+Borowiak
+Borio
+Borgos
+Borglum
+Borghoff
+Boreland
+Bordeleau
+Borchelt
+Boorman
+Boole
+Bookwalter
+Bookhart
+Bonventre
+Bonucchi
+Bonnema
+Bongard
+Bonardi
+Bonadio
+Bomstad
+Bombaci
+Bolus
+Bolognese
+Bolnick
+Bolebruch
+Boldrin
+Bolder
+Boje
+Boho
+Bohmker
+Bogosh
+Bognar
+Bogin
+Bogatitus
+Bogaert
+Boga
+Boehmke
+Boeh
+Bodway
+Bodemann
+Bockhorst
+Bochner
+Bocek
+Boblitt
+Bobbit
+Boatfield
+Boast
+Boardley
+Bo
+Blumhardt
+Blower
+Blondell
+Bloemer
+Bloczynski
+Blint
+Blenden
+Blend
+Blem
+Bleininger
+Bleile
+Blehm
+Blechman
+Bleak
+Blattler
+Blattel
+Blatherwick
+Blatchley
+Blasing
+Blasen
+Blandin
+Blaire
+Blad
+Blackler
+Bizzle
+Bison
+Bisogno
+Bisking
+Bishopp
+Bischke
+Biscaro
+Bisarra
+Birton
+Birrueta
+Birrell
+Birklid
+Binkerd
+Binetti
+Binegar
+Bindrup
+Billerbeck
+Bilka
+Biley
+Bilecki
+Biglin
+Bievenue
+Bierwagen
+Biernat
+Bienvenue
+Bielik
+Biedrzycki
+Bideaux
+Bidding
+Bickman
+Biber
+Bibel
+Biancardi
+Bialy
+Bialke
+Bialecki
+Bhattacharya
+Bezak
+Bevilaqua
+Beuth
+Beuter
+Beutel
+Beucler
+Betties
+Betteridge
+Betschart
+Betran
+Bethley
+Beteta
+Beswick
+Bessmer
+Bessemer
+Besherse
+Beserra
+Berver
+Bertuzzi
+Bertke
+Berthelsen
+Berthelette
+Bertagna
+Bersch
+Berrio
+Bernoski
+Bernatowicz
+Bernardy
+Berling
+Berl
+Bergmeier
+Bergland
+Bergfield
+Bergesen
+Bergem
+Bergantzel
+Bergamo
+Berdecia
+Berardo
+Berardino
+Bequillard
+Benzinger
+Benyamin
+Bentzen
+Bennice
+Benke
+Benet
+Beneker
+Benedum
+Benedick
+Bend
+Bencosme
+Bemrose
+Bemiller
+Bemer
+Belzung
+Belmarez
+Bellina
+Bellendir
+Bellemare
+Bellantuono
+Bellanca
+Belkin
+Belinski
+Belcourt
+Bejaran
+Behl
+Beeker
+Beeghly
+Bedney
+Bedker
+Bedeau
+Beddome
+Beddoe
+Becvar
+Beccaria
+Beaz
+Beaushaw
+Beaulac
+Beatley
+Beardon
+Beachem
+Beachel
+Bazydlo
+Baydal
+Baxi
+Bauserman
+Baudler
+Batzli
+Battino
+Battee
+Batley
+Batesole
+Batcher
+Basurto
+Basu
+Bastianelli
+Bassage
+Basner
+Bashford
+Basher
+Bashara
+Basha
+Baselice
+Bartosiewicz
+Bartolomucci
+Bartnick
+Bartholic
+Barthe
+Bartelson
+Barsuhn
+Barson
+Barries
+Barricelli
+Barrena
+Barredo
+Barraz
+Barrale
+Baroldy
+Barne
+Barmettler
+Barjas
+Baris
+Bareis
+Bardach
+Barcroft
+Barcello
+Barbuto
+Barbrick
+Barbo
+Barbish
+Barbaria
+Baras
+Baragona
+Baquet
+Banwell
+Banowetz
+Bandle
+Bambhrolia
+Balthazar
+Balson
+Balliett
+Ballestas
+Balin
+Balfany
+Balette
+Baldrige
+Baldenegro
+Baldassara
+Baldasaro
+Balcorta
+Balckwell
+Balcitis
+Balasco
+Baka
+Baish
+Bainum
+Bailin
+Baile
+Bahlmann
+Baher
+Bagoyo
+Baggette
+Bafford
+Baddley
+Badanguio
+Badamo
+Badame
+Baczewski
+Bacorn
+Bacolor
+Bacigalupi
+Bachtold
+Bacha
+Babick
+Azzano
+Azua
+Azhocar
+Ayre
+Aydt
+Aydlett
+Axsom
+Awada
+Averbach
+Avenoso
+Auzston
+Auyong
+Autaubo
+Austad
+Aus
+Aurora
+Aultz
+Aulds
+Auldridge
+Aul
+Auge
+Auel
+Audirsch
+Audain
+Auchmoody
+Aubertine
+Auber
+Astry
+Asquith
+Asp
+Ashdown
+Asen
+Aselage
+Ascensio
+Asam
+Asad
+Artuso
+Artinger
+Arritola
+Arre
+Arraiol
+Arra
+Arouri
+Arnzen
+Arntson
+Arnstein
+Arnoldy
+Arnhart
+Arnet
+Armentor
+Armel
+Arganbright
+Argall
+Argabright
+Arenstam
+Ardinger
+Arcuo
+Arambulo
+Aramboles
+Arabian
+Appelt
+Appelgren
+Apodoca
+Ape
+Anzai
+Anttila
+Antoniou
+Antoniotti
+Antonakos
+Antell
+Antee
+Antaya
+Anschutz
+Ano
+Annon
+Anne
+Annarummo
+Anick
+Angelovich
+Anes
+Androes
+Andrle
+Andreoli
+Andreassen
+Anderl
+Ancira
+Anastasi
+Anastacio
+Analla
+Ana
+Amunrud
+Amparan
+Amory
+Amores
+Amodei
+Amdahl
+Amazan
+Alway
+Alvira
+Aluise
+Altomonte
+Altidor
+Altadonna
+Alstott
+Alsina
+Alshouse
+Alpizar
+Alonge
+Almestica
+Almaras
+Almand
+Allwardt
+Allum
+Allgier
+Allerman
+Alkbsh
+Alier
+Aliano
+Alfson
+Alfero
+Alexender
+Alessandro
+Alesci
+Aldas
+Aldaba
+Alcide
+Alby
+Albelo
+Albares
+Albair
+Albach
+Alamin
+Alagna
+Akuna
+Akright
+Akim
+Akes
+Aken
+Akbari
+Akau
+Aitkins
+Aita
+Airola
+Aines
+Aimone
+Ailts
+Ahrent
+Ahne
+Ahlman
+Ahlin
+Aguire
+Agor
+Agner
+Agerter
+Age
+Agcaoili
+Afzal
+Afshari
+Affleck
+Aduddell
+Adu
+Adolfo
+Adolf
+Adjei
+Adham
+Aderholdt
+Adens
+Adee
+Adauto
+Acocella
+Ackroyd
+Ackers
+Acken
+Ack
+Achter
+Acheampong
+Aceret
+Accornero
+Abts
+Abruzzino
+Abrecht
+Abramov
+Aboud
+Abo
+Abes
+Abed
+Abby
+Aamot
+Aalbers
+Zwolensky
+Zwiener
+Zwanzig
+Zvorsky
+Zutter
+Zurowski
+Zupfer
+Zunker
+Zumbach
+Zubik
+Zubiate
+Zottola
+Zoss
+Zorman
+Zonker
+Zomer
+Zollo
+Zolezzi
+Znidarsic
+Zmijewski
+Zmich
+Zlaten
+Zisk
+Zinter
+Zingler
+Zindel
+Zimlich
+Zillman
+Zilliox
+Zigich
+Ziesemer
+Zielonka
+Ziebart
+Zia
+Zhuang
+Zeyer
+Zerkle
+Zepf
+Zenisek
+Zempel
+Zemaitis
+Zeltner
+Zellman
+Zelasco
+Zeisler
+Zeinert
+Zeier
+Zegarra
+Zeeman
+Zedaker
+Zecher
+Zeagler
+Zbinden
+Zaunbrecher
+Zarlengo
+Zannino
+Zanni
+Zangara
+Zanetti
+Zanes
+Zanderigo
+Zanayed
+Zambito
+Zalusky
+Zakutney
+Zaiss
+Zahar
+Zagrodnik
+Zaeske
+Zadroga
+Zadeh
+Zacek
+Yzaquirre
+Yuro
+Yupe
+Yunt
+Yue
+Youns
+Youngerman
+Youkhana
+Yoshizumi
+Yoshiyama
+Yoshikawa
+Yoshihara
+Yore
+Yoneda
+Yoh
+Yepsen
+Yepiz
+Yentzer
+Yelin
+Yedid
+Yeddo
+Yeboah
+Yeah
+Yauck
+Yattaw
+Yarrow
+Yarosh
+Yarn
+Yanuaria
+Yanko
+Yampolsky
+Yamin
+Yamagata
+Yakow
+Yaegle
+Yacono
+Yacko
+Xayavong
+Wythe
+Wyrich
+Wydeven
+Wyandt
+Wurtzel
+Wurdeman
+Wunner
+Wulffraat
+Wujcik
+Wry
+Wrighton
+Wreath
+Wraight
+Wragge
+Woznick
+Woten
+Wormuth
+Woofter
+Woodmore
+Woode
+Womeldorff
+Wolvin
+Wolman
+Wolgast
+Wolfgramm
+Wojtas
+Wojenski
+Wohletz
+Woetzel
+Woelke
+Woelk
+Woehrle
+Wittlinger
+Wittke
+Witthuhn
+Witthoft
+Wittekind
+Witkus
+Witbeck
+Wist
+Wissinger
+Wisnoski
+Wisley
+Wishard
+Wish
+Wipperfurth
+Winterling
+Winterholler
+Winterfeld
+Winsman
+Winkenwerder
+Wingerson
+Winegard
+Windland
+Winchel
+Wilmott
+Willwerth
+Willougby
+Willinger
+Willims
+Williby
+Willian
+Williamon
+Willhelm
+Willging
+Willens
+Willenbring
+Willcott
+Willardson
+Wilhelmy
+Wildsmith
+Wildoner
+Wildberger
+Wikholm
+Wigner
+Wiglesworth
+Wiggett
+Wiget
+Wigdor
+Wieman
+Wied
+Wieboldt
+Widen
+Wickett
+Wickard
+Wichterman
+Wichland
+Wicher
+Whysong
+Whyms
+Whooper
+Whooley
+Whitver
+Whitmoyer
+Whitehorse
+Whitebear
+Whish
+Whippo
+Wheler
+Whelehan
+Wheetley
+Wheeland
+Wheelan
+Whatoname
+Whalan
+Weygandt
+Wexell
+Wetherald
+Westfahl
+Westerholm
+Westerheide
+Westenhaver
+Westen
+Wessendorf
+Wescom
+Werstein
+Wersal
+Werra
+Werntz
+Wernicki
+Wernett
+Werger
+Werber
+Wenskoski
+Wenk
+Wendzel
+Wendelboe
+Wenciker
+Wemhoff
+Welshans
+Welde
+Welby
+Welburn
+Weisfeld
+Weisenfels
+Weinreich
+Weikert
+Weiglein
+Weida
+Wegweiser
+Wegley
+Weflen
+Weeler
+Wedo
+Wedin
+Wedgewood
+Wedderspoon
+Wedd
+Weberg
+Weathington
+Wears
+Weakly
+Weafer
+Weaber
+Waz
+Waxler
+Wave
+Wauson
+Waugaman
+Waterer
+Wasmuth
+Washmuth
+Warters
+Warsaw
+Warns
+Warnken
+Warney
+Wariner
+Warchol
+Wansitler
+Wanless
+Wanker
+Wandrie
+Wandler
+Wanczyk
+Waltmann
+Waltersdorf
+Walsworth
+Walseth
+Walp
+Walner
+Walmer
+Walloch
+Wallinger
+Wallett
+Walkley
+Walkingstick
+Walentoski
+Walega
+Wale
+Waldock
+Waldenmyer
+Walde
+Waldbauer
+Walchak
+Wakayama
+Waiau
+Waddick
+Wacyk
+Vreeken
+Vrbka
+Vradenburg
+Vounas
+Votolato
+Vosquez
+Vosika
+Vorwald
+Vorse
+Voros
+Vorgas
+Vorel
+Voorhes
+Voncannon
+Volstad
+Volo
+Volkmer
+Volden
+Volbrecht
+Voisard
+Voetsch
+Voetberg
+Voeltner
+Voegeli
+Vock
+Vlloa
+Vivona
+Vivino
+Vivenzio
+Vitucci
+Vittitoe
+Viti
+Viteaux
+Vitatoe
+Viscome
+Virzi
+Virula
+Virrey
+Virella
+Virani
+Viox
+Violetta
+Vinall
+Villatora
+Vilcan
+Vik
+Vigen
+Vieths
+Vielman
+Vidra
+Vidot
+Vidalez
+Vicent
+Vibert
+Vibbard
+Veth
+Vestering
+Veshedsky
+Versoza
+Verrell
+Veroeven
+Vernola
+Vernia
+Verjan
+Verity
+Veriato
+Verhague
+Verdusco
+Verderosa
+Verderame
+Verdell
+Verch
+Verbeke
+Venture
+Veness
+Vener
+Vendrick
+Vences
+Vellucci
+Vellone
+Velk
+Vegh
+Vedia
+Vecchiarelli
+Vazzana
+Vaux
+Vaupel
+Vaudrain
+Vatalaro
+Vastano
+Vasso
+Vasiliou
+Vasher
+Vascones
+Vas
+Varuzzo
+Varrelman
+Varnedore
+Vari
+Varel
+Vanwright
+Vanvoorhees
+Vanvolkinburg
+Vantrump
+Vanstraten
+Vanstone
+Vansice
+Vanscoter
+Vanscoit
+Vanord
+Vanoosten
+Vannortwick
+Vannette
+Vannatten
+Vanloon
+Vanliere
+Vanis
+Vanhese
+Vangalder
+Vanelderen
+Vandre
+Vandover
+Vandinter
+Vandewalle
+Vandevander
+Vanderroest
+Vandermay
+Vanderloo
+Vanderlee
+Vanderlaan
+Vandergraph
+Vanderen
+Vandenbrink
+Vandenboom
+Vandenberge
+Vandel
+Vandegriff
+Vandale
+Vanbruggen
+Vanboerum
+Vanbelle
+Vanauker
+Vanasten
+Vanarsdall
+Vallerand
+Valladao
+Valis
+Valintine
+Valenziano
+Valentia
+Valensuela
+Vaisman
+Vahena
+Vaglienty
+Vacchiano
+Uziel
+Uyemura
+Utsler
+Usie
+Urzua
+Ureste
+Urby
+Urbine
+Urabe
+Uptgraft
+Unterzuber
+Untalan
+Ungerman
+Ungerland
+Underland
+Underberg
+Umholtz
+Umbright
+Ulwelling
+Ulstad
+Ulmen
+Ulcena
+Ulanski
+Uhlenkott
+Uher
+Uhas
+Uglow
+Ugland
+Uerkwitz
+Uccellini
+Tysarczyk
+Tyron
+Twymon
+Twohey
+Twisselman
+Twichell
+Tweten
+Tuzzolo
+Tuzzo
+Tutoky
+Tusler
+Turnner
+Turja
+Turick
+Turiano
+Tunnicliff
+Tummons
+Tumlison
+Tumaneng
+Tuder
+Tuczynski
+Tuchman
+Tubville
+Tsukiyama
+Tselee
+Truxon
+Truxler
+Trussler
+Trusler
+Trusillo
+Trudillo
+Trude
+Truchan
+Trowery
+Trotochaud
+Tropiano
+Tronstad
+Trolinger
+Trocinski
+Triveno
+Trites
+Triplet
+Trick
+Trichell
+Trichel
+Trevey
+Trester
+Treisch
+Treger
+Trefz
+Tredwell
+Trebbe
+Treakle
+Travillion
+Travillian
+Travaglio
+Trauscht
+Traube
+Trapper
+Tranum
+Trani
+Train
+Towlson
+Towlerton
+Towey
+Tovmasyan
+Tousley
+Tourtellotte
+Toure
+Toulson
+Totin
+Tosti
+Tosado
+Toruno
+Torrisi
+Torris
+Torrent
+Torrado
+Torner
+Torino
+Torell
+Topolansky
+Tooze
+Toot
+Tontarski
+Tonnessen
+Tonneson
+Tones
+Tomisin
+Tomilson
+Tomasetti
+Tolomeo
+Tollman
+Tolhurst
+Tolchin
+Tolbent
+Toher
+Toffton
+Toepel
+Toelkes
+Todorovich
+Todisco
+Toczek
+Tockey
+Tochterman
+Tobiasson
+Tlucek
+Titzer
+Titman
+Tise
+Tippets
+Tio
+Tingwald
+Timmel
+Timbrook
+Tilmon
+Tijerino
+Tigerino
+Tigano
+Tieken
+Tiegs
+Tiefenbrun
+Tichacek
+Tica
+Thurmer
+Thuotte
+Thramer
+Thoroughman
+Thornock
+Thorndyke
+Thongchanh
+Thomen
+Thoe
+Thody
+Thigpin
+Thielemier
+Thi
+Therres
+Thal
+Thakur
+Tewes
+Teves
+Tesmer
+Teslow
+Tesler
+Teruel
+Terron
+Terris
+Terre
+Terrasi
+Terrace
+Tero
+Terman
+Tereska
+Teresi
+Tepp
+Teo
+Tenzer
+Tennille
+Tennies
+Tencza
+Tenamore
+Tejadilla
+Tecklenburg
+Techaira
+Tayse
+Tawwater
+Tavolacci
+Taverner
+Taurino
+Taulman
+Taublee
+Tauarez
+Tattershall
+Tatsuta
+Tatsuno
+Taschner
+Tasby
+Tarrats
+Tarrants
+Tarone
+Tarley
+Taraborelli
+Taper
+Tanniehill
+Tanks
+Tankard
+Tangri
+Tanequodle
+Tamporello
+Tamer
+Tamburro
+Tambunga
+Taliman
+Talib
+Talas
+Takala
+Takach
+Taiwo
+Taibi
+Taghon
+Tagaban
+Tadena
+Taccone
+Taccetta
+Tabatabai
+Szyszka
+Szmalc
+Szerszen
+Szczepanik
+Szarek
+Szafraniec
+Szafran
+Szablewski
+Syta
+Sysyn
+Syndergaard
+Symanski
+Sylvian
+Syck
+Swymer
+Swoffer
+Swoager
+Swiggum
+Swiat
+Swetnam
+Swestka
+Swentzel
+Sweetwood
+Swedenburg
+Swearingin
+Swartzendrube
+Swarm
+Swant
+Swancey
+Sverchek
+Svenson
+Sutor
+Suthoff
+Suthar
+Susong
+Suskin
+Surra
+Surano
+Supplee
+Supino
+Sundborg
+Summons
+Summerour
+Sumers
+Sultzer
+Sulouff
+Sulecki
+Suhoski
+Suhar
+Sugerak
+Suganuma
+Suddoth
+Sudberry
+Sud
+Stymiest
+Stvrestil
+Stuve
+Sturrup
+Sturmer
+Stumer
+Stuhlsatz
+Stuenkel
+Studier
+Stuczynski
+Stubbolo
+Struebing
+Struchen
+Strozzi
+Strowder
+Strohbehn
+Stroer
+Strobridge
+Strobeck
+Stritmater
+Strike
+Strieter
+Strickling
+Streu
+Streifel
+Straugter
+Stratakos
+Strasburger
+Straface
+Straatmann
+Stpeters
+Stovel
+Stoudenmire
+Stotsky
+Stothart
+Storz
+Stormes
+Storman
+Stoppel
+Stooks
+Stonelake
+Stonebrook
+Stombaugh
+Stoltzman
+Stolsig
+Stolpe
+Stoglin
+Stoffle
+Stodgell
+Stocke
+Stirna
+Stipetich
+Stinner
+Stimpert
+Stimer
+Stilphen
+Stikeleather
+Stifel
+Stiely
+Stielau
+Stieger
+Stidman
+Stickrath
+Stickman
+Stickels
+Stgerard
+Sternberger
+Stergios
+Stepien
+Stepanski
+Stent
+Stenkamp
+Stenehjem
+Stempel
+Stemmer
+Stelb
+Steiskal
+Steinmuller
+Steinmacher
+Steinhorst
+Steinhaus
+Steinharter
+Steinhagen
+Steinburg
+Steifle
+Stefanick
+Stefanich
+Steeber
+Stay
+Stawarz
+Stavropoulos
+Staves
+Staup
+Stauch
+Staubs
+Stathopoulos
+Stathis
+Startz
+Starowitz
+Starowicz
+Starkie
+Starcic
+Stanely
+Standrod
+Standahl
+Stanczak
+Stample
+Stampka
+Stamer
+Stallins
+Stalford
+Stahoski
+Stagger
+Stader
+Staack
+Srsic
+Srey
+Squitieri
+Spyres
+Spuhler
+Sprouffske
+Sprosty
+Sprinzl
+Springle
+Spoth
+Spletzer
+Spizer
+Spitsberg
+Spitale
+Spiroff
+Spirer
+Spiotta
+Spinola
+Spingler
+Spike
+Spierling
+Spickler
+Sphon
+Spettel
+Sperle
+Sperka
+Sperberg
+Speltz
+Spaw
+Spasiano
+Spare
+Spancake
+Spagna
+Sowerby
+Sovern
+Souvannasap
+Southerly
+Sous
+Sourwine
+Soult
+Sotiriou
+Sothman
+Sota
+Sortore
+Sorley
+Sorin
+Sorells
+Soratos
+Soose
+Soong
+Sonsino
+Sonnabend
+Sonia
+Songster
+Sondrol
+Sondergaard
+Soltau
+Solinski
+Solinger
+Solid
+Sojda
+Sohns
+Softleigh
+Soffel
+Soffa
+Sodaro
+Sodano
+Soda
+Sobran
+Sobczynski
+Sneeden
+Snater
+Snair
+Smoker
+Smithingell
+Smink
+Smiles
+Smialek
+Smetak
+Smejkal
+Smeck
+Smaldone
+Sluyter
+Slot
+Slostad
+Slingerland
+Sliffe
+Slemmer
+Slawter
+Slavinski
+Slagowski
+Slaff
+Skuse
+Skulski
+Skornia
+Skolfield
+Skogstad
+Skinkle
+Skidgel
+Skeffington
+Skeets
+Skeele
+Skarupa
+Skarphol
+Skaare
+Sjolander
+Sjaarda
+Sitts
+Sitterud
+Sitt
+Sissell
+Siprasoeuth
+Sipper
+Sipla
+Sipkema
+Sinning
+Sinitiere
+Single
+Simmens
+Simm
+Simiskey
+Simelton
+Silverthorne
+Silvernale
+Silvan
+Siliado
+Silbaugh
+Siket
+Siker
+Sigurdson
+Signore
+Sigers
+Siffert
+Sieving
+Sieverding
+Sietsema
+Siering
+Sienicki
+Siemsen
+Siemonsma
+Siemering
+Sielski
+Siedlecki
+Siebers
+Sidbury
+Sickman
+Sickinger
+Sicilian
+Sible
+Sibilio
+Sibble
+Shutler
+Shurgot
+Shuping
+Shulda
+Shula
+Shrieves
+Shreiner
+Shreckengost
+Shreck
+Showes
+Showe
+Shoupe
+Shoumaker
+Shortey
+Shorten
+Shorrock
+Shorkey
+Shones
+Shockency
+Shoats
+Shivel
+Shipmen
+Shinsel
+Shindledecker
+Shinabarger
+Shiminski
+Shiloh
+Shillingford
+Shigo
+Shifman
+Shiers
+Shibuya
+Shewchuk
+Shettsline
+Shetter
+Shetrawski
+Sheffel
+Sheesley
+Sheekey
+Sheeder
+Sheares
+Shauger
+Sharko
+Shanna
+Shankin
+Shani
+Shandley
+Shanaa
+Shammo
+Shamlin
+Shambrook
+Shadow
+Shackley
+Sgambati
+Sferrazza
+Seydel
+Sewald
+Sevenbergen
+Sevaaetasi
+Seumanu
+Seuell
+Settler
+Setterberg
+Setera
+Sesso
+Sesay
+Servoss
+Servino
+Serpe
+Sermeno
+Serles
+Serena
+Serapio
+Senske
+Semmler
+Seminole
+Semel
+Selvaggi
+Sellai
+Selissen
+Seling
+Seleg
+Seledon
+Selbo
+Selan
+Sekuterski
+Sekula
+Seiwell
+Seivert
+Seise
+Sein
+Seils
+Seier
+Seidita
+Seiberling
+Seher
+Segroves
+Segoviano
+Segel
+Segee
+Seftick
+Sees
+Seekell
+Seegobin
+Seebold
+Sedlack
+Sedbrook
+Section
+Secrease
+Secore
+Seckler
+Seastrand
+Seargent
+Seacrist
+Seachord
+Seabrooke
+Scudieri
+Scrim
+Scozzafava
+Scotten
+Sconce
+Scircle
+Scipioni
+Sciarretta
+Sciallo
+Schwingler
+Schwinghammer
+Schwingel
+Schwiesow
+Schweinfurth
+Schweda
+Schwebke
+Schwarzkopf
+Schwander
+Schwaller
+Schwall
+Schut
+Schurkamp
+Schunter
+Schulder
+Schuenemann
+Schue
+Schuckman
+Schuchart
+Schroff
+Schoville
+Schorzman
+Schorder
+Schooner
+Schones
+Scholler
+Schofell
+Schoewe
+Schoeninger
+Schoenhals
+Schoenbeck
+Schoefield
+Schoberg
+Schnittker
+Schneidermann
+Schneckloth
+Schnebly
+Schnathorst
+Schnarrs
+Schnakenberg
+Schmitzer
+Schmidbauer
+Schmeeckle
+Schmeckpeper
+Schmandt
+Schmalzried
+Schmal
+Schlinker
+Schliep
+Schlette
+Schlesier
+Schleig
+Schlehuber
+Schlarbaum
+Schlaffer
+Schkade
+Schissel
+Schindeldecke
+Schimandle
+Schiermeier
+Scheunemann
+Scherrman
+Schepp
+Schemmer
+Schelp
+Schehr
+Schayer
+Schaunaman
+Schauland
+Schatzel
+Scharrer
+Scharping
+Scharpf
+Scharnberg
+Scharmer
+Scharbor
+Schalow
+Schaf
+Schader
+Schacter
+Scelfo
+Scarpello
+Scarlet
+Scaringe
+Scarduzio
+Scamardo
+Scaman
+Sbano
+Sayman
+Saylee
+Saxena
+Sawdey
+Sawada
+Savitsky
+Savickas
+Savic
+Savaglio
+Sauriol
+Sauret
+Saulo
+Satar
+Sasportas
+Sarvas
+Sarullo
+Sarsfield
+Sarne
+Sarmento
+Sarjent
+Sarellano
+Sardin
+Saputo
+Santheson
+Santellana
+Santarsiero
+Santago
+Sansalone
+Sanos
+Sanna
+Sanko
+Sanker
+Sanghani
+Sangalli
+Sandven
+Sandmann
+Sandhoff
+Sandelius
+Sandall
+Sanchious
+Sancedo
+Sance
+Sampogna
+Sampilo
+Sampayan
+Sampaia
+Sampaga
+Samo
+Samlal
+Samela
+Samec
+Samad
+Salzberg
+Salway
+Salwasser
+Salveson
+Salvemini
+Salus
+Salquero
+Salowitz
+Salizzoni
+Salina
+Salin
+Salimi
+Salgero
+Salemi
+Salato
+Salassi
+Salamacha
+Salahubdin
+Salada
+Saintignon
+Saintamand
+Saines
+Sahl
+Saha
+Sagona
+Sagedahl
+Saffel
+Saemenes
+Sadow
+Sadlow
+Sadger
+Sacramento
+Sackal
+Sachtleben
+Sabota
+Sabot
+Sabe
+Sabata
+Sabastian
+Sabad
+Rzepka
+Ryzinski
+Rytuba
+Ryon
+Rynes
+Rykiel
+Rykert
+Rykard
+Rydolph
+Rydell
+Ruzicki
+Rutko
+Rutenbar
+Rustrian
+Rusinski
+Rushmore
+Rushenberg
+Rushen
+Ruschak
+Rury
+Ruper
+Ruotolo
+Rummerfield
+Rumer
+Rumbolt
+Rulon
+Ruleman
+Rufe
+Rudo
+Rudkin
+Rudick
+Rubinich
+Rubidoux
+Rubero
+Roys
+Rowman
+Rovere
+Rousu
+Rouillier
+Rotton
+Rotondi
+Rothenbach
+Roszell
+Rossotto
+Rossmiller
+Rossey
+Roshannon
+Rosenfeldt
+Roscioli
+Rosander
+Rorrer
+Rorex
+Ropes
+Ropac
+Rooth
+Roorda
+Ronsani
+Ronne
+Rong
+Ronfeldt
+Rondy
+Romp
+Romon
+Romness
+Romm
+Romera
+Romeiro
+Rombach
+Romar
+Romansky
+Romagnoli
+Rom
+Rolson
+Rojos
+Rohanna
+Rogstad
+Rogillio
+Rogg
+Rogacki
+Roffman
+Roethle
+Roeth
+Roetcisoender
+Rodibaugh
+Roderiques
+Rodenburg
+Rodemeyer
+Rodberg
+Rockovich
+Rocher
+Roccio
+Robeck
+Robe
+Robayo
+Robar
+Rizzardo
+Rivie
+Rival
+Ritterbush
+Ritchko
+Ritchhart
+Ristig
+Rishty
+Rippstein
+Rippelmeyer
+Rioseco
+Ringwald
+Ringquist
+Ringham
+Rinella
+Rineer
+Rimple
+Rilling
+Rill
+Rijo
+Riihimaki
+Riglos
+Riggens
+Rigaud
+Rigali
+Rietz
+Rietdorf
+Riessen
+Riesgraf
+Rienstra
+Riekena
+Riedle
+Riedinger
+Rieb
+Rickenbaker
+Richcreek
+Richbourg
+Riccelli
+Riberdy
+Ribb
+Rhodie
+Rheome
+Rheinhardt
+Rezai
+Reynalds
+Reyman
+Reyez
+Rewenko
+Reville
+Revello
+Revelez
+Reul
+Resue
+Restuccia
+Replenski
+Reon
+Rentar
+Rensberger
+Rens
+Rennaker
+Renell
+Remson
+Rell
+Relacion
+Rekuc
+Reker
+Reitler
+Reischl
+Reints
+Reinoehl
+Reinart
+Reimund
+Reimold
+Reikowsky
+Reiger
+Reifman
+Reicks
+Reichler
+Reichhardt
+Rehling
+Regos
+Regino
+Regalbuto
+Reffner
+Reents
+Reenders
+Reeks
+Reek
+Reeck
+Redmer
+Redican
+Reddoch
+Reddig
+Reddicks
+Redbird
+Rectenwald
+Recek
+Rebillard
+Rebich
+Rebeck
+Reagon
+Raziano
+Raymore
+Ravenel
+Ravel
+Rause
+Rauschenbach
+Rauer
+Rauchwerger
+Ratelle
+Rasinski
+Rasbury
+Rardon
+Rapson
+Rapkin
+Raoof
+Rannells
+Ranke
+Rangitsch
+Rangasammy
+Randt
+Ran
+Ramser
+Ramsaroop
+Ramsahai
+Ramrez
+Rampley
+Ramirec
+Ramesh
+Ralbovsky
+Rakoczy
+Rakoci
+Rajwani
+Rajaratnam
+Raiden
+Rahmani
+Ragno
+Raghunandan
+Ragas
+Ragar
+Rafuse
+Radvany
+Rados
+Radmacher
+Radick
+Radecki
+Raczynski
+Rachell
+Qureshi
+Quirin
+Quire
+Quintona
+Quinnett
+Quinalty
+Quiambao
+Quella
+Quatraro
+Quartararo
+Qualle
+Qin
+Pytko
+Pyer
+Pyanowski
+Puzio
+Pushcar
+Purviance
+Purtlebaugh
+Pupo
+Pulte
+Pulse
+Pullom
+Pullings
+Pullano
+Pulkkinen
+Puliafico
+Pulfrey
+Pujols
+Puhala
+Puchalla
+Pucciarelli
+Prutzman
+Prutt
+Pruneau
+Prucha
+Provitt
+Protin
+Prose
+Proco
+Proa
+Prisk
+Prioletti
+Priode
+Prinkey
+Princiotta
+Prich
+Pribnow
+Prial
+Preyer
+Prestino
+Pressimone
+Preskitt
+Preli
+Preissler
+Prehoda
+Predovich
+Precise
+Prazenica
+Prawdzik
+Prast
+Pozzobon
+Pozos
+Powles
+Pov
+Poullard
+Pouch
+Potucek
+Postert
+Posten
+Posson
+Posa
+Portuondo
+Porten
+Porst
+Poree
+Pora
+Poque
+Popiolek
+Poot
+Poock
+Pongkhamsing
+Ponessa
+Pone
+Poncio
+Polumbo
+Pollutro
+Pollet
+Pollen
+Poljak
+Polemeni
+Pokswinski
+Poisel
+Poette
+Poelman
+Pody
+Podewils
+Podaras
+Pocius
+Pobanz
+Plympton
+Ply
+Plush
+Plume
+Pluff
+Plues
+Plue
+Plona
+Plexico
+Plew
+Pleiss
+Pleil
+Pleasanton
+Plattsmier
+Plathe
+Plankey
+Plahs
+Plagge
+Placker
+Placha
+Pizira
+Piwowar
+Piwetz
+Pittelkow
+Pitta
+Pithan
+Pitcherello
+Pisciotti
+Pipilas
+Pintea
+Pinta
+Pinkstaff
+Pinkos
+Pinc
+Pilotte
+Pillo
+Pihl
+Pignotti
+Piggs
+Pietrzyk
+Piermont
+Pieczynski
+Piechowski
+Piech
+Pickersgill
+Picetti
+Picciuto
+Piccinini
+Picarello
+Picardo
+Picado
+Piantanida
+Pianka
+Pian
+Phothirath
+Phippard
+Philman
+Philipson
+Philavanh
+Phelts
+Phanor
+Phanco
+Pflughoeft
+Pflugh
+Pfliger
+Pfeister
+Pfeifle
+Peyre
+Peyatt
+Pettine
+Pettett
+Petru
+Petronio
+Petricka
+Petrak
+Petko
+Petitto
+Petersson
+Pesnell
+Peshek
+Pesh
+Pescador
+Perze
+Perteet
+Pertee
+Pert
+Perschbacher
+Perruzzi
+Perrish
+Perrigan
+Perriello
+Perr
+Perozo
+Perlich
+Perking
+Perkes
+Perfater
+Perce
+Pepez
+Peon
+Penunuri
+Penuel
+Penso
+Pennisi
+Penkins
+Penkalski
+Pendon
+Pellon
+Pellissier
+Pelino
+Pel
+Peick
+Peguese
+Peggs
+Pefanis
+Peeters
+Peedin
+Peduto
+Pedulla
+Pedrozo
+Pedrotti
+Pedroncelli
+Pedrogo
+Pedri
+Pedregon
+Pederzani
+Pedde
+Pecukonis
+Peckler
+Pecka
+Pecha
+Pecci
+Peatman
+Peals
+Pazo
+Paye
+Pawlusiak
+Pawlitschek
+Pavlosky
+Pavlo
+Paveglio
+Paulman
+Paukstis
+Pauk
+Patts
+Patter
+Patriss
+Patneaude
+Paszek
+Paswaters
+Pastula
+Pastuch
+Pastel
+Passy
+Passarella
+Pasquin
+Pasqualetti
+Pasqual
+Pascuzzi
+Pasceri
+Parviainen
+Parral
+Parolini
+Parmele
+Parma
+Parlavecchio
+Parfitt
+Parez
+Pardieck
+Pardew
+Parda
+Paraz
+Parat
+Papay
+Paparello
+Papaioannou
+Paolello
+Pansini
+Panelli
+Panell
+Pander
+Pancholi
+Panaro
+Panagiotopoul
+Palomarez
+Palmrose
+Palmisciano
+Palmese
+Pallotto
+Palleschi
+Palk
+Palhegyi
+Palenzuela
+Paleaae
+Palczynski
+Palakiko
+Palaia
+Paith
+Pagonis
+Pago
+Pagliuca
+Pagliari
+Paganini
+Padovani
+Padfield
+Padamadan
+Pacquette
+Paco
+Packwood
+Pachero
+Pachar
+Pacewicz
+Paasch
+Pa
+Ozols
+Ozga
+Ozenne
+Oxman
+Overpeck
+Overbeek
+Overbee
+Oulette
+Otsu
+Otremba
+Otool
+Otar
+Otanicar
+Osumi
+Osucha
+Ostrov
+Osthoff
+Ostertag
+Ostergard
+Ostaba
+Ospital
+Ososkie
+Osofsky
+Osisek
+Oshinsky
+Orzalli
+Orwin
+Ortwein
+Ortuno
+Orts
+Ortell
+Orpen
+Ornelaz
+Orewiler
+Ores
+Ordones
+Opunui
+Oppenlander
+Opoien
+Opalka
+Ooley
+Ontko
+Ondrey
+Omura
+Omtiveros
+Omland
+Olup
+Olthoff
+Olsten
+Ollila
+Olivia
+Olinsky
+Olinick
+Oleksa
+Olejarz
+Oldakowski
+Okoronkwo
+Okins
+Ohmer
+Ohlsson
+Oherron
+Oheron
+Ohanian
+Oganesian
+Ogaldez
+Oest
+Oehlenschlage
+Oedekerk
+Odon
+Odekirk
+Ocran
+Oconor
+Obrzut
+Obrist
+Obringer
+Oborny
+Oblander
+Obi
+Oberley
+Oberer
+Obeng
+Oatridge
+Oajaca
+Nypaver
+Nuzzi
+Nuzback
+Nuxoll
+Nussbaumer
+Nurmi
+Nuhn
+Nugen
+Nuara
+Nquyen
+Nozicka
+Noxon
+Nowick
+Nowaczyk
+Novielli
+Novembre
+November
+Novas
+Noun
+Notto
+Notowich
+Norzagaray
+Norway
+Northover
+Northcross
+Norem
+Nordmann
+Nordenson
+Nolet
+Nojiri
+Nohel
+Noethiger
+Nodd
+Nitzel
+Nita
+Nisbit
+Nina
+Nikas
+Nigon
+Niglio
+Nighswander
+Nighbert
+Niemietz
+Niedzielski
+Niederkorn
+Niederhaus
+Niederer
+Nicometo
+Nicolaides
+Nickolich
+Nguyn
+Neyra
+Neymeyer
+Newmon
+Newgent
+Newbery
+Nevala
+Neuweg
+Neuhoff
+Neuhauser
+Neubecker
+Nettik
+Netters
+Nestingen
+Nesspor
+Nerad
+Nenez
+Neldon
+Neizer
+Neives
+Neils
+Neiger
+Neidich
+Neibert
+Negroni
+Neemann
+Needle
+Neeb
+Nedry
+Nedley
+Neas
+Naze
+Nazaroff
+Nayes
+Nayar
+Nattress
+Natonabah
+Nassr
+Nasseri
+Nassef
+Naso
+Narkier
+Naret
+Nardini
+Nardecchia
+Naragon
+Naputi
+Napierala
+Nanny
+Nanke
+Namdar
+Naji
+Naidoo
+Nahm
+Nahas
+Nagelschmidt
+Naes
+Naegeli
+Nacol
+Naclerio
+Nachor
+Nabozny
+Nabarrete
+Nab
+Myrlie
+Mykins
+Muzio
+Mutolo
+Muta
+Mustoe
+Muster
+Muske
+Muschamp
+Muscarello
+Musacchio
+Murzycki
+Murrufo
+Murnan
+Muraski
+Murany
+Murano
+Munzer
+Munis
+Munion
+Mumby
+Mumbower
+Mulrain
+Mullinex
+Mullineaux
+Mullennix
+Mullahey
+Mukhtar
+Muina
+Muha
+Muehlman
+Muccigrosso
+Mrozoski
+Mozier
+Mow
+Mova
+Moustafa
+Mousser
+Mouse
+Mousa
+Mouritsen
+Mourad
+Mottet
+Motten
+Motamedi
+Mostowy
+Mostafavi
+Mosiman
+Moscone
+Moscicki
+Mosbrucker
+Morva
+Mortinez
+Mortel
+Morsey
+Morrin
+Morren
+Morosco
+Morledge
+Morla
+Morisky
+Morishita
+Morisey
+Morgia
+Moretta
+Morera
+Morenz
+Mordue
+Mordhorst
+Mordaunt
+Morber
+Morawa
+Moravick
+Morarity
+Mooty
+Mooser
+Moock
+Moochler
+Montoure
+Montooth
+Montonez
+Montierth
+Monticello
+Monteverde
+Monterrano
+Montella
+Montecillo
+Monsrud
+Monsma
+Monserrat
+Monrreal
+Monro
+Monetti
+Mondok
+Mondella
+Moncion
+Monaldi
+Moltz
+Molon
+Mollicone
+Molle
+Moliterno
+Molinere
+Molinary
+Molesworth
+Moh
+Mogush
+Mogren
+Moellers
+Moeck
+Modert
+Mockbee
+Mocher
+Mochel
+Moc
+Moberley
+Moan
+Moallankamp
+Miyose
+Miyata
+Miyashita
+Miyagi
+Mitsuda
+Misumi
+Missel
+Miskelly
+Misiaszek
+Mirzadeh
+Mirto
+Mirsch
+Mirles
+Miolen
+Minzel
+Minutillo
+Minugh
+Mintzer
+Minskey
+Minnaert
+Minkoff
+Miniard
+Mingledorff
+Minas
+Minaai
+Milly
+Millinor
+Millie
+Millerd
+Millea
+Milkey
+Milham
+Milfeld
+Mileham
+Milas
+Milar
+Milak
+Mikulski
+Mihara
+Mihalek
+Mihalchik
+Mihal
+Mignot
+Mignano
+Mighty
+Miesse
+Mierzwinski
+Micthell
+Mickus
+Mickolick
+Mickiewicz
+Michlin
+Michelena
+Micha
+Miccio
+Micari
+Mezzatesta
+Mewbourn
+Meuse
+Meurin
+Metzker
+Mettling
+Metting
+Metters
+Metropoulos
+Metevia
+Mesteth
+Mesko
+Mesi
+Meserole
+Mervyn
+Mernin
+Mermelstein
+Merling
+Merli
+Merkowitz
+Merklin
+Merkerson
+Merica
+Merendino
+Mercury
+Meray
+Meranto
+Merancio
+Mensik
+Mense
+Menoni
+Mennie
+Mengsteab
+Menes
+Mend
+Mency
+Memolo
+Meltz
+Meling
+Melen
+Melcer
+Melamed
+Mekee
+Meiste
+Meise
+Meinhard
+Meierotto
+Mehok
+Meharg
+Meginnes
+Meenach
+Medicus
+Mediano
+Media
+Medell
+Mede
+Meddaugh
+Meconi
+Mech
+Mearse
+Meardon
+Mealor
+Meadville
+Meachen
+Mcvicar
+Mcsparin
+Mcrorie
+Mcrobbie
+Mcoy
+Mcowen
+Mcnorton
+Mcnertney
+Mcnamer
+Mcnail
+Mcmanamon
+Mcmain
+Mclyman
+Mcleland
+Mckirgan
+Mckew
+Mckevitt
+Mckercher
+Mckensie
+Mckeegan
+Mckeane
+Mckahan
+Mcinture
+Mcindoe
+Mcilvenny
+Mcillwain
+Mciff
+Mcgwin
+Mcguff
+Mcgrotty
+Mcgrone
+Mcgrant
+Mcgoogan
+Mcglon
+Mcgloin
+Mcgiveron
+Mcghehey
+Mcghay
+Mcgavin
+Mcgahen
+Mcfann
+Mcelwaine
+Mcelduff
+Mceachron
+Mcdilda
+Mcdermid
+Mcdannold
+Mcdale
+Mcculough
+Mccuien
+Mccrumb
+Mccrorey
+Mccreless
+Mccravy
+Mccourtney
+Mccorrison
+Mccorkell
+Mccorey
+Mcconney
+Mcconnaughhay
+Mccollester
+Mcclurkan
+Mccluer
+Mccloudy
+Mcclenaghan
+Mcclave
+Mcclarnon
+Mcclarin
+Mcclaney
+Mcclanan
+Mcclair
+Mcchristion
+Mccaskell
+Mccartha
+Mccarl
+Mccamant
+Mccalmont
+Mccalman
+Mccaine
+Mccahill
+Mccague
+Mcbrown
+Mcanany
+Mcalvain
+Mazzurco
+Mazuc
+Mazo
+Mazingo
+Mawhorter
+Mavro
+Mavraganis
+Mautner
+Mautino
+Mauceli
+Matzinger
+Maturi
+Matturro
+Mattlin
+Mattheis
+Matsuoka
+Matsuki
+Matro
+Matlack
+Matice
+Mathson
+Matheu
+Mathenia
+Math
+Matejka
+Mateja
+Matanane
+Masztal
+Mastropaolo
+Mastromarino
+Mastrolia
+Mastel
+Massy
+Massoud
+Massimino
+Maslanka
+Masini
+Mascioli
+Marzec
+Marvier
+Maruyama
+Marusarz
+Marum
+Martorella
+Martire
+Martinkus
+Martinas
+Martiez
+Marthe
+Marteney
+Marschall
+Marruffo
+Marrazzo
+Marples
+Marohl
+Marn
+Marlborough
+Markunas
+Marki
+Marjan
+Maritnez
+Marinkovic
+Marineau
+Margaitis
+Marentis
+Mare
+Marcou
+Marciel
+Marci
+Marchiori
+Marchello
+Marchell
+Marcelle
+Marcelin
+Marales
+Mapel
+Manzanarez
+Mantilia
+Mansmith
+Manon
+Mannschreck
+Mannick
+Mankiewicz
+Mankel
+Manila
+Manifold
+Manha
+Mangrich
+Mangiapane
+Mangiamele
+Manera
+Mandes
+Mandella
+Mandelik
+Mandaloniz
+Mand
+Mancusi
+Mancine
+Mana
+Mamula
+Mammoccio
+Malzhan
+Malzahn
+Malsom
+Maloon
+Malnar
+Mallone
+Mallinson
+Mallie
+Mallek
+Malle
+Malinoski
+Malinconico
+Malicoat
+Malicdem
+Malhi
+Malfatti
+Malandrino
+Malamud
+Malakowsky
+Makovec
+Makey
+Majercik
+Majer
+Majamay
+Maisenbacher
+Mainey
+Mailey
+Mailander
+Mahuna
+Mahomes
+Mahoe
+Mahnken
+Maheras
+Mahaxay
+Mahana
+Maham
+Magnia
+Magni
+Magnanti
+Magliano
+Magliacane
+Maglaughlin
+Magistrale
+Magierski
+Maggini
+Magano
+Mafnas
+Madren
+Mador
+Maderios
+Madena
+Maddron
+Madan
+Madalinski
+Macmanus
+Maclead
+Mackowski
+Mackinaw
+Mackessy
+Mackerl
+Macker
+Macivor
+Machold
+Machain
+Macedonio
+Macdiarmid
+Macchiaroli
+Macbean
+Macayan
+Macari
+Mabin
+Mabel
+Lyter
+Lyster
+Lysne
+Lynskey
+Lyness
+Lyndaker
+Lymaster
+Lykke
+Lyell
+Luxmore
+Luttmer
+Lutgen
+Lusignan
+Lupold
+Lungstrom
+Lunford
+Lundeby
+Lumbard
+Lule
+Lukaskiewicz
+Luinstra
+Luevand
+Luer
+Lueking
+Luehrs
+Luecking
+Ludvigson
+Ludgood
+Lucich
+Luchetti
+Lubman
+Lubic
+Lozito
+Lowhorn
+Lowd
+Loverich
+Loveman
+Lovas
+Lovaas
+Louvier
+Louthen
+Loury
+Loukanis
+Loughner
+Loughnane
+Louato
+Lotshaw
+Lother
+Lothamer
+Loter
+Losinski
+Losinger
+Loshek
+Losecco
+Lortie
+Lorin
+Lorent
+Lorello
+Loras
+Lorah
+Lopau
+Loosen
+Lontz
+Longpre
+Longie
+Loncaric
+Lombrana
+Lomba
+Lohrey
+Lohoff
+Logghe
+Loges
+Lofstead
+Lofft
+Loertscher
+Loeper
+Loeblein
+Lodato
+Lochen
+Lobbins
+Lobban
+Lizarrago
+Livigni
+Livernash
+Liukko
+Littich
+Litterer
+Littau
+Litchmore
+Lisy
+Lissy
+Lishman
+Lischak
+Lirag
+Liptow
+Lins
+Linkhart
+Linkert
+Lingren
+Lingelbach
+Lingel
+Lingad
+Linet
+Linegar
+Linebrink
+Lindroth
+Lindeland
+Lindboe
+Linardi
+Linard
+Ligman
+Liggans
+Lifland
+Liff
+Lieuallen
+Liesveld
+Liess
+Lienhard
+Liehr
+Liedy
+Liedke
+Liebau
+Lidtke
+Lidstrom
+Licano
+Libra
+Leys
+Leymeister
+Lewerke
+Lewand
+Levoci
+Leviton
+Levien
+Leveston
+Leverenz
+Levere
+Levangie
+Leuy
+Leukuma
+Lettman
+Letran
+Letlow
+Lethco
+Letersky
+Lestronge
+Lesso
+Lessey
+Leshem
+Lerud
+Leps
+Leonesio
+Leones
+Lento
+Lente
+Lennertz
+Lenior
+Lenhard
+Lenfest
+Lene
+Lendrum
+Lempicki
+Lemonier
+Lemle
+Lemkau
+Lemings
+Lem
+Lelli
+Lekas
+Leitten
+Leitheiser
+Leino
+Leiner
+Leinenbach
+Leidy
+Leidich
+Leid
+Leich
+Lehnhoff
+Leh
+Legum
+Legoullon
+Legeyt
+Legalley
+Legace
+Lefton
+Lefthand
+Leforge
+Lefore
+Lefleur
+Leerar
+Leef
+Leed
+Ledl
+Leddon
+Ledain
+Leckie
+Lecates
+Lebeouf
+Leben
+Lebeck
+Lebeaux
+Leban
+Leaverton
+Learman
+Leardi
+Leamy
+Lazare
+Lazarczyk
+Layssard
+Layson
+Layhew
+Layel
+Laychock
+Lawernce
+Lavzon
+Lavalla
+Lauterborn
+Laut
+Lauseng
+Lausen
+Laurino
+Lauri
+Laurenzano
+Laurenza
+Laundry
+Laumbach
+Lauinger
+Lauenroth
+Latzke
+Latulipe
+Lattig
+Latronica
+Latouf
+Latko
+Latiker
+Lathern
+Laterza
+Latchaw
+Lataquin
+Lasure
+Lashomb
+Lasell
+Lasasso
+Lartey
+Larriva
+Laro
+Lardner
+Lardieri
+Laprarie
+Lapping
+Lapitan
+Lapeyrolerie
+Lapar
+Lanzetta
+Lantis
+Lanka
+Lani
+Langshaw
+Langmyer
+Langin
+Langerman
+Langeland
+Langbein
+Landro
+Landrian
+Landmesser
+Landmann
+Landfair
+Landesberg
+Lanciotti
+Lamprey
+Lampey
+Lamos
+Lamora
+Lamoine
+Lamfers
+Lambka
+Lamance
+Lamana
+Laliotis
+Lajza
+Lajaunie
+Lainson
+Laher
+Lahar
+Lagrotta
+Lagrant
+Lagraize
+Lagnese
+Lafrazia
+Lafountaine
+Laflin
+Lafaso
+Lafarga
+Ladage
+Lacsamana
+Lacrosse
+Lacrone
+Lachowski
+Labruyere
+Labrake
+Labossiere
+Laba
+Laack
+Kyzar
+Kynard
+Kwek
+Kuzmin
+Kuttner
+Kusiak
+Kuser
+Kuse
+Kurtzer
+Kurtzeborn
+Kurpinski
+Kurohara
+Kuroda
+Kurnik
+Kurihara
+Kurdziel
+Kurban
+Kuras
+Kupper
+Kupferer
+Kupec
+Kunzelman
+Kunkler
+Kunin
+Kunesh
+Kumro
+Kumpf
+Kulon
+Kulka
+Kukucka
+Kuk
+Kuhse
+Kuhls
+Kuhlo
+Kuhar
+Kuerbitz
+Kuenzi
+Kuehneman
+Kudron
+Kuczenski
+Kuchle
+Kuchenmeister
+Kuchenbecker
+Kucan
+Kubu
+Kubsch
+Kubiszewski
+Kubish
+Kubicz
+Kubick
+Kubaska
+Kuarez
+Ksiazek
+Kshywonis
+Krzykowski
+Krzak
+Krysl
+Kruzewski
+Kruzan
+Krumrine
+Krumins
+Krucker
+Kroupa
+Krough
+Krotz
+Kronstedt
+Kromrey
+Krogstad
+Krogmann
+Kroeze
+Kroetz
+Kroc
+Kristianson
+Kristen
+Kriser
+Krips
+Kringas
+Kriete
+Kreuter
+Kretschmann
+Kresha
+Kreidel
+Kregger
+Kreatsoulas
+Kratochwil
+Krasovec
+Krase
+Krapf
+Kranawetter
+Krajnik
+Kozubal
+Koyanagi
+Kowalkowski
+Kovarovic
+Kovalcin
+Kou
+Kotzen
+Kotnik
+Kostelecky
+Kostek
+Kostecki
+Kostal
+Kosse
+Koslowski
+Koskie
+Kosicki
+Koshar
+Kosek
+Kortright
+Korpal
+Kornhauser
+Kormos
+Korinek
+Korgie
+Kordsmeier
+Kordish
+Koral
+Kops
+Kopps
+Kopperud
+Koppang
+Kopfer
+Kopet
+Kook
+Konno
+Konik
+Konek
+Konefal
+Komm
+Komis
+Komer
+Komarek
+Kolsrud
+Kolp
+Kolopajlo
+Kollmorgen
+Kolis
+Kolesnik
+Koles
+Kolding
+Kohs
+Kohlhoff
+Kohatsu
+Kohara
+Koetter
+Koestler
+Koepsel
+Koeppe
+Koenigsman
+Koelewyn
+Koe
+Kodadek
+Koci
+Kochler
+Kocab
+Kobylinski
+Kobryn
+Koberg
+Knower
+Knollenberg
+Knock
+Knizley
+Kniss
+Knies
+Knezovich
+Knesek
+Knepel
+Knehans
+Kneeskern
+Knaust
+Knapke
+Kmet
+Kluz
+Klukas
+Kloska
+Klopf
+Klinglesmith
+Klinekole
+Klimes
+Kliment
+Klimaszewski
+Klepfer
+Klepacki
+Klepac
+Klemash
+Kleinkopf
+Kleinknecht
+Kleimola
+Kleiboeker
+Klei
+Klehn
+Klegin
+Klavuhn
+Klauer
+Klasinski
+Klasing
+Klarr
+Klapec
+Klaass
+Klaameyer
+Kjelland
+Kiyuna
+Kitching
+Kistle
+Kissi
+Kishi
+Kirvin
+Kirtner
+Kirovac
+Kirnon
+Kirkby
+Kiritsy
+Kirchgesler
+Kippley
+Kipping
+Kinzig
+Kins
+Kinnare
+Kinna
+Kingcade
+Kinatyan
+Kimme
+Kimbrow
+Kimbril
+Kilzer
+Kiltz
+Killmer
+Killibrew
+Killeagle
+Kilger
+Kiles
+Kievit
+Kientzy
+Kielty
+Kiekbusch
+Kiehne
+Kiefert
+Khou
+Khiev
+Khat
+Khare
+Keywan
+Keyt
+Kevin
+Keville
+Kevern
+Keuler
+Ketola
+Ketelaar
+Kertis
+Kerson
+Kernen
+Kerkman
+Kerker
+Keogan
+Kenwood
+Kenne
+Kenaan
+Kempler
+Kempisty
+Kempfer
+Kempen
+Kemmerlin
+Kelter
+Kelman
+Kellie
+Keliihoomalu
+Keleman
+Kekiwi
+Keiswetter
+Keiss
+Keilty
+Keidong
+Kegel
+Keets
+Keeneth
+Keefner
+Kedzierski
+Kebort
+Keate
+Keat
+Kazmorck
+Kazi
+Kaz
+Kawachi
+Kaushiva
+Kauk
+Katzner
+Katzmark
+Katzen
+Katsuda
+Kats
+Kater
+Katen
+Kasting
+Kasserman
+Kassay
+Kassabian
+Kasprowicz
+Kasperek
+Kasowski
+Kasmir
+Kaska
+Kasik
+Kascak
+Karth
+Karsnak
+Karshner
+Karsh
+Karmel
+Karlstad
+Karley
+Karins
+Karimi
+Karcich
+Karch
+Karapetyan
+Karakas
+Kapsalis
+Kappeler
+Kapke
+Kaperonis
+Kapahu
+Kanthak
+Kansky
+Kansas
+Kanoy
+Kanno
+Kannady
+Kandarian
+Kanai
+Kanae
+Kanaan
+Kamphoefner
+Kammler
+Kaminetzky
+Kaminaka
+Kamienski
+Kamaunu
+Kamakea
+Kama
+Kaltefleiter
+Kaloustian
+Kaloi
+Kallmeyer
+Kalisch
+Kalinski
+Kaliher
+Kalgren
+Kalfas
+Kales
+Kalafatis
+Kagle
+Kadish
+Kachermeyer
+Kabina
+Kaawa
+Kaaua
+Kaatz
+Juvera
+Jutte
+Justen
+Jusko
+Juriga
+Jure
+Jungquist
+Jungbluth
+Juneja
+Juncaj
+Juliet
+Juhas
+Juenger
+Juell
+Jucean
+Jubinville
+Jovich
+Jorres
+Joris
+Jore
+Jonhson
+Joneson
+Jonassen
+Jolissaint
+Jointer
+Johnny
+Johengen
+Johar
+Joh
+Joern
+Jodway
+Jobs
+Joanette
+Jirik
+Jirasek
+Jipson
+Jinkerson
+Jinkens
+Jiminian
+Jimeno
+Jiau
+Jevnikar
+Jessel
+Jerauld
+Jephson
+Jentzen
+Jenkerson
+Jenista
+Jenifer
+Jemmett
+Jelovich
+Jehlicka
+Jeffris
+Jedziniak
+Jeantet
+Jeanclaude
+Jayme
+Javor
+Javaux
+Jaurigue
+Jaureguy
+Jarvinen
+Jarocki
+Japp
+Janszen
+Jansons
+Jans
+Jankauskas
+Janka
+Janhunen
+Janeczek
+Jandrin
+Janczewski
+Janack
+Jamir
+Jakuboski
+Jakubik
+Jakubek
+Jahnel
+Jageman
+Jaenicke
+Jacquem
+Jacquay
+Jaconski
+Jacobellis
+Jablon
+Iyo
+Ivancevic
+Iurato
+Iulianetti
+Itri
+Issler
+Isla
+Isip
+Ishmon
+Ishizu
+Isgrigg
+Iseri
+Iseli
+Iseley
+Isbrecht
+Isassi
+Isaiah
+Irsik
+Irias
+Inzana
+Intveld
+Intrieri
+Interdonato
+Instasi
+Inscho
+Ingwell
+Ingebretsen
+Inga
+Inda
+Incle
+Inabinett
+Imus
+Immordino
+Imbesi
+Imbach
+Illsley
+Illig
+Ill
+Ignowski
+Idler
+Idleburg
+Ideue
+Ibara
+Ianuzzi
+Ianniello
+Iacovone
+Hyter
+Hyles
+Hyle
+Hykes
+Hyams
+Huxley
+Hutch
+Hustead
+Huscher
+Hurtz
+Hurse
+Hurren
+Huret
+Huotari
+Huntress
+Hunting
+Hunstiger
+Hunking
+Humpries
+Humbles
+Hum
+Hulvey
+Hulcy
+Huizinga
+Huhman
+Huhammad
+Hufty
+Huesso
+Hueftle
+Huebschman
+Huebert
+Hue
+Hudmon
+Huberman
+Hubbartt
+Hubach
+Hsueh
+Hrycenko
+Hrabal
+Hoxit
+Howsare
+Howman
+Howitt
+Howerter
+Houlton
+Houis
+Hottman
+Hotovec
+Hostin
+Hoshall
+Hosfeld
+Hoschek
+Horwath
+Horsely
+Horsburgh
+Horovitz
+Hornstrom
+Hornbarger
+Horkley
+Horka
+Horey
+Horeth
+Hordyk
+Horack
+Hoppin
+Hoppel
+Hopfensperger
+Hooey
+Hooe
+Honhart
+Honga
+Honeck
+Homs
+Hommell
+Homles
+Homen
+Home
+Holzner
+Holzheimer
+Holzem
+Holsopple
+Holsman
+Holowell
+Holliway
+Holizna
+Holesovsky
+Holderbaum
+Holbach
+Holan
+Hoit
+Hoist
+Hohenbrink
+Hoger
+Hofmans
+Hofheimer
+Hoffhines
+Hofbauer
+Hoesing
+Hoeschen
+Hoerter
+Hoepfner
+Hoemann
+Hodgeman
+Hockersmith
+Hochadel
+Hobock
+Hobel
+Hluska
+Hlavac
+Hisrich
+Hirsbrunner
+Hirpara
+Hire
+Hinners
+Hindbaugh
+Himenez
+Hilles
+Hilleary
+Hillanbrand
+Hillan
+Hildner
+Hilding
+Hilderbrandt
+Hiland
+Hightree
+Highnote
+Highberger
+Higgason
+Higaneda
+Hidinger
+Hickock
+Heymann
+Heusinkveld
+Heusel
+Heuring
+Hettler
+Hesseltine
+Hesselink
+Hesford
+Herth
+Herskovits
+Herschell
+Heroman
+Hernton
+Herne
+Hernandaz
+Hermez
+Hermanstorfer
+Herling
+Herke
+Herimann
+Heriford
+Hergenrader
+Herforth
+Herdes
+Hercher
+Herceg
+Herbick
+Hentze
+Henniger
+Henney
+Henness
+Hennegan
+Henkes
+Heneisen
+Henderickson
+Henard
+Hemrick
+Hemric
+Hempton
+Hemp
+Hemme
+Hemeon
+Hembry
+Hembrough
+Hembrey
+Helstad
+Helmus
+Hellings
+Hellgren
+Helie
+Helgert
+Helgerman
+Helger
+Helgason
+Helfinstine
+Helfgott
+Helfenstein
+Heldreth
+Helander
+Heitzmann
+Heisserer
+Heising
+Heisel
+Heinold
+Heinis
+Heinemeyer
+Heimark
+Heiliger
+Heiderman
+Heidenescher
+Heidebrink
+Hehir
+Hegan
+Heersink
+Heep
+Hedquist
+Heckford
+Hebets
+Heberly
+Heberle
+Hebenstreit
+Heavilin
+Heartz
+Heaphy
+Heany
+Hazer
+Hazelgrove
+Haynsworth
+Haydock
+Hawelu
+Havnen
+Havely
+Hauss
+Hausam
+Haumesser
+Hauman
+Haulk
+Hauley
+Haubrick
+Haubner
+Hattman
+Hatman
+Hatherly
+Hatchcock
+Hastert
+Hassenplug
+Hasko
+Haser
+Haselhuhn
+Hasberry
+Has
+Harthorne
+Harthcock
+Harriett
+Harouff
+Harootunian
+Harkavy
+Harell
+Hardridge
+Hardacre
+Harborth
+Haraguchi
+Haptonstall
+Happenny
+Hantman
+Hanses
+Hannemann
+Hannay
+Hannafin
+Hanle
+Hangartner
+Handerson
+Hanberg
+Hamzik
+Hamstra
+Hammans
+Hamano
+Halsema
+Halonen
+Halim
+Halek
+Haleamau
+Halama
+Hakeem
+Hainley
+Hagley
+Hagist
+Hagie
+Haggberg
+Haggan
+Hagele
+Hafenstein
+Hafemeister
+Hady
+Hadges
+Hadef
+Hackey
+Hach
+Habbyshaw
+Haaga
+Haab
+Gysin
+Gwirtz
+Guzzio
+Guzzardo
+Guzma
+Gutzmann
+Gutta
+Gutermuth
+Guterman
+Gutenberger
+Gurganious
+Gural
+Guppy
+Gunzalez
+Guntert
+Gums
+Gumb
+Gullotta
+Gullixson
+Gulling
+Gullace
+Guler
+Gulbransen
+Guitian
+Guinta
+Guinasso
+Guilboard
+Guichard
+Gugliotta
+Guglielmina
+Guggenheim
+Gugel
+Guetierrez
+Guethle
+Gueth
+Guerrido
+Gueits
+Gudenkauf
+Gucciardo
+Guarnera
+Guadagnolo
+Gsell
+Gschwend
+Grush
+Grupp
+Grundmann
+Grunau
+Grueninger
+Gruca
+Groupe
+Grotzinger
+Grotheer
+Grossmeyer
+Grossetete
+Grossack
+Gromer
+Groenke
+Groening
+Groehler
+Groebner
+Grochmal
+Groby
+Grobes
+Gritman
+Griswould
+Grisset
+Grime
+Griffo
+Griesinger
+Greuel
+Greth
+Gressman
+Gremel
+Greiwe
+Greis
+Greil
+Greife
+Greider
+Grefrath
+Greff
+Greenmyer
+Greany
+Grazioplene
+Gravlin
+Gravito
+Gravert
+Grav
+Grater
+Grap
+Granzin
+Grannum
+Granlund
+Grando
+Grammes
+Gramley
+Grambo
+Grala
+Grahl
+Gradwohl
+Gradillas
+Gradert
+Graciana
+Grabner
+Grabinski
+Grabinger
+Grabel
+Graaf
+Gouzy
+Gouger
+Gottron
+Gottardo
+Gothro
+Gosso
+Gossi
+Gorringe
+Gorneault
+Gorn
+Gormly
+Gorenflo
+Goral
+Gopen
+Goosey
+Goodnoe
+Goodie
+Goodhile
+Goodfield
+Goodard
+Gonneville
+Gongalez
+Gondola
+Gompf
+Gommer
+Gollehon
+Golie
+Golebiewski
+Goldinger
+Goldhaber
+Goldfeder
+Goldbaum
+Golaszewski
+Gojcaj
+Gogerty
+Goettsche
+Goethe
+Goessl
+Godson
+Godbe
+Gochanour
+Gocha
+Gnau
+Gnatek
+Glud
+Glorius
+Glordano
+Gloodt
+Glod
+Glinka
+Glime
+Gleim
+Gleicher
+Glazewski
+Glay
+Glasford
+Glascott
+Glanzman
+Glahn
+Gladish
+Gjerde
+Gizinski
+Gitzen
+Girsh
+Girote
+Girman
+Giovino
+Giovanini
+Giorgini
+Ginty
+Ginsky
+Ginnings
+Gingues
+Gingg
+Ginger
+Giner
+Gimm
+Gilruth
+Gillund
+Gillenwaters
+Gilday
+Gilcrest
+Gilcher
+Gilani
+Gigstad
+Giernoth
+Gienger
+Gidaro
+Giczewski
+Gibas
+Giarratano
+Giantonio
+Giannitti
+Giannetti
+Giampapa
+Giacopelli
+Giacone
+Giacomelli
+Gherman
+Ghera
+Ghan
+Gevorkyan
+Gettig
+Getchman
+Gesinski
+Gerundo
+Gershenson
+Gerraro
+Gernert
+Germundson
+Gerloff
+Gergel
+Gerdeman
+Gerdel
+Geraldo
+Geraldes
+Georgopoulos
+Georgis
+Georgevic
+Georgeson
+Genzel
+Genung
+Gentzler
+Gentili
+Genich
+Gelzinis
+Geiken
+Geidner
+Geidl
+Gehrer
+Geho
+Gehlbach
+Geeding
+Gedye
+Geberth
+Geathers
+Gearan
+Gealy
+Gazzola
+Gazella
+Gawrych
+Gavidia
+Gautam
+Gaumont
+Gaudenzi
+Gaucher
+Gaubert
+Gattas
+Gatley
+Gaters
+Gatchalian
+Gassel
+Gasman
+Gaslin
+Garufi
+Garriepy
+Garrell
+Garrand
+Garnto
+Garns
+Garno
+Garlinger
+Garivay
+Garhart
+Gardino
+Garcea
+Garbin
+Garaventa
+Garavaglia
+Garahan
+Garafano
+Garacia
+Gapen
+Ganiron
+Ganino
+Ganim
+Gangwish
+Gange
+Ganes
+Gandia
+Gandeza
+Gamlin
+Gamelin
+Galway
+Galow
+Gallob
+Gallishaw
+Gallinaro
+Gallicchio
+Gallese
+Gallero
+Gallegas
+Galeoto
+Galeas
+Galbreth
+Galbavy
+Galavis
+Galam
+Gajate
+Gair
+Gagney
+Gagel
+Gagarin
+Gaete
+Gaetani
+Gadbaw
+Gack
+Gabrysch
+Gabardi
+Fyksen
+Futrelle
+Furl
+Furches
+Furbeck
+Funnye
+Funicello
+Fumagalli
+Fullford
+Fulginiti
+Fulenwider
+Fulena
+Fugler
+Fuerstenberge
+Fuentas
+Fucillo
+Fuapau
+Fryberger
+Frusciante
+Fruehling
+Fromberg
+Froeschle
+Frock
+Fritzgerald
+Fritcher
+Frisbey
+Frihart
+Frieling
+Friedler
+Frie
+Fridell
+Freuden
+Freud
+Frett
+Frend
+Freiling
+Freije
+Freie
+Freidman
+Freibert
+Fregozo
+Freehling
+Fredo
+Fredlund
+Fredley
+Frede
+Freberg
+Frayre
+Fraunfelter
+Frascella
+Franssen
+Frankowski
+Francour
+Francom
+Francillon
+Francey
+Fraioli
+Fracassa
+Fostervold
+Fossey
+Foshay
+Foscue
+Forsell
+Forrister
+Forren
+Fornicola
+Fornes
+Forgie
+Forbs
+Foppe
+Foore
+Fontecchio
+Fongeallaz
+Follick
+Folio
+Foder
+Flyzik
+Fluhman
+Fluet
+Flow
+Floto
+Floros
+Floriano
+Floren
+Floran
+Floerke
+Flitcroft
+Flipp
+Flintroy
+Fleschner
+Flenner
+Fleeting
+Flamio
+Flaggs
+Flagge
+Fjeseth
+Fithen
+Fissell
+Fischman
+Fire
+Fioranelli
+Finseth
+Finocchiaro
+Finerty
+Fineman
+Finchman
+Filyaw
+Filipovich
+Filas
+Figler
+Figge
+Fiers
+Fiereck
+Fidell
+Ficorilli
+Fico
+Ficks
+Fickle
+Fialkowski
+Feyen
+Fetz
+Fetsko
+Ferullo
+Fertitta
+Ferriman
+Ferrebee
+Ferrand
+Ferrales
+Fernelius
+Fernberg
+Ferioli
+Fergoson
+Ferenc
+Fereira
+Fequiere
+Fennema
+Fenelus
+Fenelon
+Feneis
+Femrite
+Feltenberger
+Felsenthal
+Fels
+Felmet
+Felgenhauer
+Felarca
+Feiteira
+Feirer
+Feinen
+Feigenbaum
+Fehlinger
+Federle
+Fecko
+Feavel
+Featheringham
+Fayer
+Faxon
+Faurrieta
+Faull
+Fatone
+Fatigate
+Fasy
+Fasula
+Fassio
+Fass
+Farwick
+Farrill
+Farquer
+Farmwald
+Fantozzi
+Fanoele
+Fannell
+Fanizza
+Fandrich
+Fallo
+Fallago
+Faist
+Faines
+Faine
+Fahrendorff
+Faggard
+Faessler
+Fadale
+Fabrizi
+Eychaner
+Exon
+Exilus
+Ewig
+Evitts
+Evinger
+Everheart
+Everhardt
+Eveleth
+Eveleigh
+Eurbin
+Esworthy
+Estus
+Estock
+Esterbrook
+Essler
+Esque
+Espina
+Espalin
+Eschenburg
+Eschberger
+Esbenshade
+Ertley
+Erstad
+Erp
+Eroman
+Erno
+Ermatinger
+Erkkila
+Erkela
+Eriquez
+Erin
+Ericks
+Erdahl
+Ercolani
+Equils
+Eppinette
+Eon
+Enter
+Enke
+Engley
+Englebrecht
+Engleberg
+Englar
+Engelstad
+Engelsman
+Engellant
+Ence
+Emslie
+Empie
+Emoto
+Emons
+Emley
+Emile
+Embly
+Embler
+Emanuelson
+Emal
+Elzinga
+Elwer
+Elvis
+Elvington
+Elshere
+Elmquist
+Ellout
+Ellifritz
+Ellerd
+Ellerbusch
+Elizando
+Elizabeth
+Elick
+Eliasen
+Elgert
+Elger
+Elena
+Elbers
+Ekstein
+Ekmark
+Eiser
+Einck
+Eimers
+Eilert
+Eidinger
+Eicke
+Ehsan
+Ehn
+Egleton
+Egel
+Effner
+Ednilao
+Edner
+Edmons
+Edmister
+Edmison
+Edlow
+Edholm
+Edgeman
+Edgcomb
+Edell
+Edelblute
+Eclarinal
+Eckroad
+Echave
+Ebesu
+Eberwein
+Ebeid
+Ebe
+Ebbing
+Eastlund
+Eary
+Earps
+Dzuro
+Dziuban
+Dysinger
+Dyner
+Dymek
+Dyll
+Dyl
+Dydell
+Dwelle
+Dwan
+Duvernois
+Dutson
+Dutro
+Dutchover
+Dusky
+Duskey
+Dusik
+Dushkin
+Dushane
+Durrani
+Duroseau
+Durnford
+Durk
+Durepo
+Duranceau
+Duprat
+Duplechin
+Duperry
+Dunscomb
+Dunkleberger
+Dung
+Dunegan
+Dundlow
+Dumpson
+Dumphy
+Dumpert
+Dumesnil
+Dullum
+Duldulao
+Dular
+Dukart
+Duhan
+Dugdale
+Dugat
+Duffney
+Duesing
+Duenow
+Duce
+Dubson
+Drzewicki
+Druetta
+Drube
+Drozdenko
+Drop
+Drohan
+Drivers
+Drinski
+Driever
+Drewer
+Dressen
+Drehmer
+Drawe
+Drapkin
+Draney
+Drahota
+Dowers
+Dowdall
+Dovenbarger
+Dousay
+Douin
+Doughan
+Doucett
+Douce
+Dorshimer
+Dorsaint
+Dorries
+Dorosky
+Dorl
+Dorich
+Dorenfeld
+Dorcelus
+Dool
+Donoso
+Donnick
+Donnely
+Donart
+Donalds
+Donaghey
+Donaghe
+Dominges
+Domebo
+Dollings
+Dolejsi
+Doggette
+Doell
+Dockwiller
+Dockal
+Dobosh
+Dobis
+Dobiesz
+Dluhy
+Dixons
+Divin
+Diventura
+Divenere
+Divelbiss
+Dittrick
+Ditommaso
+Dirosa
+Dircks
+Diogo
+Diodonet
+Dinning
+Dininno
+Dimodica
+Dimitroff
+Diminno
+Dimassimo
+Dillie
+Dilan
+Digsby
+Digrande
+Digmann
+Digirolomo
+Digian
+Digiacinto
+Dietzen
+Dietlin
+Dietert
+Diersen
+Dienst
+Dieffenbach
+Dicorcia
+Dickhaut
+Diberardino
+Diab
+Dhein
+Dhar
+Dhamer
+Dezan
+Dez
+Dewispelaere
+Dewhirst
+Devonish
+Devincenzo
+Devillez
+Devany
+Devalcourt
+Deubler
+Dettori
+Detone
+Detommaso
+Detoma
+Desue
+Destree
+Destephen
+Desso
+Desselle
+Desimoni
+Desadier
+Derham
+Derfler
+Dercole
+Derasmo
+Depugh
+Deporter
+Depolito
+Depa
+Deninno
+Deni
+Denenberg
+Denaro
+Denardis
+Demry
+Demro
+Demmel
+Demme
+Demiel
+Demeritte
+Demarzio
+Demaline
+Demaine
+Deluco
+Delton
+Delsordo
+Delosa
+Delongis
+Delois
+Deloff
+Delmuro
+Delmoro
+Delmonaco
+Delmage
+Dellen
+Dellaripa
+Dellamore
+Delhierro
+Delfuente
+Deleppo
+Delemos
+Delea
+Delcarmen
+Delaura
+Delanuez
+Delang
+Delamarter
+Delamare
+Delage
+Delacuesta
+Dekorte
+Dekenipp
+Dekany
+Deinhardt
+Deily
+Deierlein
+Degravelle
+Deglow
+Degler
+Degiulio
+Defoore
+Defonce
+Deflorio
+Defiore
+Defilippi
+Deed
+Dedeke
+Dedecker
+Dedaj
+Decost
+Decillis
+Dechellis
+Dechaine
+Decarr
+Decaprio
+Debutiaco
+Debski
+Debry
+Debruhl
+Debouse
+Deblase
+Debey
+Debenedetti
+Debacker
+Deang
+Deandrade
+Deadmond
+Deacy
+Daykin
+Dayhuff
+Dayal
+Davion
+Davidsen
+Dautremont
+Daughrity
+Daubs
+Datwyler
+Datko
+Dasmann
+Daruszka
+Darugar
+Darroch
+Daro
+Darkis
+Daricek
+Daras
+Dar
+Dapoz
+Dapinto
+Danuser
+Danoff
+Dankmeyer
+Danesi
+Danesh
+Daneker
+Dammen
+Damien
+Damberger
+Dalmoro
+Dallmier
+Daller
+Dalka
+Daliva
+Dahline
+Dahlhauser
+Daguerre
+Dagrella
+Dagraca
+Dagesse
+Dage
+Daehn
+Dado
+Dabbraccio
+Dabato
+Czolba
+Czepiel
+Czelusniak
+Czechowski
+Czarny
+Czar
+Czapski
+Cywinski
+Cyran
+Cypret
+Cwiek
+Cuzzort
+Cuzzi
+Cutty
+Cutrone
+Cuthrell
+Cuthill
+Cutbirth
+Custeau
+Cushingberry
+Curvey
+Curson
+Currell
+Curly
+Curll
+Curdy
+Curcuru
+Cupstid
+Cuoco
+Culverson
+Culnane
+Culliver
+Cullivan
+Culleton
+Cuddeback
+Cuckler
+Cubillo
+Cubias
+Cua
+Cryar
+Crutsinger
+Crusan
+Crupe
+Crummie
+Cruice
+Cruea
+Crowthers
+Crowers
+Crowdis
+Crovo
+Croson
+Crosno
+Crosdale
+Cronwell
+Cronon
+Crocetti
+Crnich
+Cristal
+Crisson
+Crismond
+Crighton
+Cridland
+Crickard
+Creten
+Cretella
+Crespino
+Cremins
+Cremers
+Creehan
+Creecy
+Credell
+Cranney
+Cranker
+Craker
+Craffey
+Cozzy
+Coyazo
+Coxum
+Cowdin
+Covino
+Coven
+Courtenay
+Course
+Courier
+Courchene
+Coup
+Couley
+Couchenour
+Cotugno
+Cottongim
+Cotti
+Cotillo
+Costine
+Costain
+Cosmo
+Coslan
+Cose
+Coryea
+Cortwright
+Corsoro
+Corrente
+Correl
+Cornford
+Corneluis
+Cornelious
+Corneau
+Corne
+Corkins
+Corippo
+Corgiat
+Coreil
+Cordwell
+Cordovano
+Cordill
+Cordano
+Corazza
+Coran
+Coppess
+Coonrad
+Coonfare
+Coomber
+Cooksley
+Cookis
+Coodey
+Contrino
+Contee
+Consorti
+Console
+Conorich
+Conole
+Connoly
+Connley
+Connington
+Connie
+Conness
+Conly
+Conkright
+Coner
+Conchas
+Comrie
+Compston
+Compagno
+Comnick
+Commiskey
+Commer
+Comiso
+Comish
+Comden
+Colondres
+Collica
+Colleen
+Colle
+Collaer
+Colinger
+Colford
+Colao
+Colanero
+Cohens
+Cofresi
+Coerver
+Cockriel
+Cockran
+Cockerell
+Cobham
+Cobert
+Cobern
+Cobell
+Clunie
+Clubs
+Clubbs
+Cloutman
+Clise
+Clippinger
+Clerkley
+Cler
+Clemmens
+Clemen
+Cleare
+Cleamons
+Claycamp
+Clawges
+Claverie
+Clarkston
+Clarity
+Clantz
+Clakley
+Clain
+Cizek
+Ciuffreda
+Citrone
+Ciraco
+Cinotto
+Cini
+Cinadr
+Cilento
+Cilano
+Cihon
+Ciganek
+Cieslinski
+Cicoria
+Cicco
+Cibula
+Ciarrocchi
+Ciak
+Ciafardoni
+Chubbs
+Chrzan
+Christophel
+Christoph
+Christoforou
+Christel
+Christan
+Chreene
+Chrabaszcz
+Chrabasz
+Chowhan
+Choules
+Chorney
+Chorley
+Cholico
+Cholewinski
+Cholakyan
+Chojnowski
+Chlebek
+Chittam
+Chiszar
+Chisam
+Chirafisi
+Chiprean
+Chinetti
+Chimes
+Chiera
+Chicon
+Chiarelli
+Chiaravalle
+Chiappetta
+Chesner
+Cheser
+Chesbrough
+Cherubino
+Cherrette
+Cherpak
+Chelf
+Cheesebrough
+Cheeney
+Cheely
+Chean
+Cheak
+Chavana
+Chauvette
+Chatt
+Chasser
+Chaskey
+Charriez
+Chappie
+Chappelear
+Chapparo
+Chapek
+Chanoine
+Chandley
+Challenger
+Challberg
+Challacombe
+Chaleun
+Chainey
+Chaffey
+Cetta
+Cerza
+Cervenak
+Certosimo
+Cerruti
+Cerqueira
+Cernohous
+Cereceres
+Ceovantes
+Ceo
+Centrich
+Centore
+Cellucci
+Ceglinski
+Ceconi
+Cecilio
+Cecchinato
+Cecchi
+Cazorla
+Cayne
+Cayabyab
+Cavill
+Cavicchia
+Cavez
+Cavener
+Cavasos
+Cavaness
+Cavalcante
+Caulk
+Caudel
+Cattano
+Catrett
+Catlow
+Catella
+Cataquet
+Catalino
+Cataline
+Catalanotto
+Catalanatto
+Cata
+Castenanos
+Castelo
+Cassiday
+Casparian
+Casillo
+Casewell
+Casarrubias
+Casalman
+Casal
+Carvalno
+Carskadon
+Carrus
+Carrison
+Carriker
+Carrazco
+Carratala
+Carpanini
+Carovski
+Caroli
+Carne
+Carmella
+Carlis
+Carfagno
+Carethers
+Carella
+Cardonia
+Cardno
+Carda
+Carcieri
+Carcano
+Carcana
+Carboneau
+Carbon
+Caravantes
+Carattini
+Caramanica
+Capriola
+Cappelluti
+Capossela
+Caponi
+Caperon
+Caper
+Capati
+Cantv
+Cantore
+Cantell
+Cantatore
+Cantarella
+Cantadore
+Canslor
+Canonico
+Cannonier
+Cannone
+Cannavo
+Cannatella
+Cangiano
+Campoli
+Campellone
+Campean
+Campanile
+Camera
+Camcam
+Cambel
+Calta
+Callsen
+Callarman
+Calicott
+Calhaun
+Calegari
+Calco
+Calciano
+Calabretta
+Cake
+Cairone
+Cahela
+Cagliostro
+Caflisch
+Cafferky
+Caetano
+Cadice
+Caddle
+Cadarette
+Cackowski
+Caccia
+Cabrena
+Cabotaje
+Caborn
+Caberto
+Bystrom
+Byndon
+Buzek
+Buysse
+Bux
+Buttrick
+Buttaro
+Butscher
+Butsch
+Butor
+Butman
+Buteux
+Butchee
+But
+Bustard
+Busta
+Bussy
+Busson
+Bussing
+Bussa
+Busi
+Buseman
+Buschner
+Buscaglia
+Burttram
+Burth
+Bursch
+Burnsworth
+Burland
+Burkowski
+Burglin
+Burgdorfer
+Burdman
+Burau
+Buran
+Burakowski
+Buquet
+Buonomo
+Buntyn
+Bungo
+Bunche
+Bunal
+Bult
+Bulliner
+Bullaro
+Bulkeley
+Bulcao
+Bula
+Buisson
+Buissereth
+Bugni
+Buetow
+Buesgens
+Budziszewski
+Budinich
+Buddington
+Buchtel
+Buchli
+Buchert
+Buchar
+Buben
+Brzuchalski
+Brummell
+Brull
+Brudnicki
+Brucz
+Bruchman
+Brubach
+Brownwood
+Browen
+Browe
+Brossett
+Brosco
+Brookshear
+Brookfield
+Bronstad
+Bronsky
+Bronaugh
+Bron
+Brohawn
+Brogna
+Brodzik
+Brodsho
+Brodowski
+Brodnicki
+Brodell
+Brod
+Brockney
+Broas
+Broadrick
+Briz
+Britschgi
+Brint
+Brinich
+Bringard
+Brindamour
+Brincat
+Brimfield
+Brillant
+Brilhante
+Brihon
+Brignoni
+Brightful
+Briggman
+Bried
+Brickle
+Brickel
+Brezeale
+Brewen
+Breutzman
+Bretado
+Brester
+Bresko
+Brennon
+Brennaman
+Breniser
+Brendon
+Brems
+Breisch
+Breidenstein
+Brechtel
+Brea
+Brazington
+Brazen
+Brayer
+Brawer
+Bravata
+Braune
+Braunbeck
+Braue
+Braucht
+Braseth
+Brantly
+Branter
+Branski
+Brandler
+Bramham
+Brahney
+Bradac
+Brackley
+Brackey
+Brackemyre
+Brach
+Boyarsky
+Bowlan
+Bowhall
+Bowdre
+Bovie
+Bouyea
+Boustead
+Bourgeault
+Bounthapanya
+Boultinghouse
+Bouillon
+Boudrie
+Boudinot
+Bottgenbach
+Bottari
+Botos
+Bothof
+Botha
+Bosten
+Bostelmann
+Bossley
+Bossick
+Bossen
+Bosquet
+Boscio
+Bosche
+Bosa
+Borski
+Borsh
+Borowik
+Borom
+Borke
+Borgerding
+Borgatti
+Bordwine
+Booser
+Bookbinder
+Bookard
+Boock
+Bonte
+Bonomi
+Bonning
+Bonito
+Bonillas
+Bondura
+Bombich
+Boltinghouse
+Bollozos
+Bolliger
+Bollie
+Bolka
+Bolitho
+Boldenow
+Bolch
+Bolay
+Boissoneault
+Boisjolie
+Boisclair
+Boie
+Bohrman
+Bohley
+Boglioli
+Boghosian
+Boggus
+Boggiano
+Bogden
+Boey
+Boesenhofer
+Boerst
+Boerma
+Boenisch
+Boemig
+Boebinger
+Boday
+Bodamer
+Bocklage
+Bocchini
+Bobseine
+Bobian
+Boberg
+Bobek
+Blyler
+Blumenstein
+Bloyer
+Blotter
+Blore
+Blomme
+Blomdahl
+Bliske
+Blinston
+Bliek
+Blessman
+Bleggi
+Bleeker
+Bledsaw
+Blauch
+Blaskovich
+Blankley
+Blankenberg
+Blanken
+Blakelock
+Blaida
+Bjorgen
+Biven
+Bitzel
+Bittman
+Bitonti
+Bissen
+Bisom
+Bisher
+Birman
+Birky
+Birkes
+Bippus
+Bintz
+Bintner
+Bintliff
+Binnie
+Binks
+Binkiewicz
+Binienda
+Bingley
+Bilotto
+Billheimer
+Billen
+Billeck
+Billeaudeau
+Bilinski
+Bilello
+Bild
+Bihari
+Bigda
+Biez
+Bierwirth
+Bierle
+Bierbower
+Bienenstock
+Biemer
+Bieler
+Bielak
+Bidle
+Biddleman
+Biddiscombe
+Bicknese
+Bickerton
+Bickelhaupt
+Bichsel
+Bibles
+Bibian
+Biase
+Biancuzzo
+Biancaniello
+Biamonte
+Bia
+Bhatnagar
+Bhardwaj
+Bhan
+Beyett
+Bewig
+Beuchat
+Better
+Betsill
+Bethey
+Betenbaugh
+Betance
+Betacourt
+Beske
+Besendorfer
+Besemer
+Besco
+Bery
+Bertran
+Bertling
+Bertie
+Bernson
+Bernosky
+Bernon
+Berninger
+Bernes
+Bernecker
+Bernasconi
+Bernardin
+Berlo
+Berliew
+Berky
+Berhe
+Berhalter
+Bergsjo
+Bergholm
+Bergener
+Bergeman
+Beraun
+Benward
+Benusa
+Bense
+Bennage
+Benischek
+Benion
+Beninato
+Bengel
+Benedek
+Bene
+Bendzus
+Bendler
+Bendit
+Benderman
+Benberry
+Benallie
+Bemrich
+Belyea
+Beltrain
+Belter
+Bellue
+Bellocchio
+Bellisle
+Bellipanni
+Bellion
+Bellessa
+Bellavia
+Belay
+Bejjani
+Beisser
+Beiriger
+Beik
+Beien
+Behymer
+Behrenwald
+Behanna
+Beed
+Beechum
+Beechner
+Bednarik
+Bednarek
+Bedenbaugh
+Becwar
+Beckton
+Beckom
+Bech
+Bebo
+Beatie
+Beat
+Bearman
+Beaner
+Beakley
+Beahan
+Beachamp
+Bazzi
+Bayman
+Bayardo
+Bayala
+Bawcum
+Bavier
+Bauswell
+Baures
+Baune
+Baumgarter
+Bault
+Baughey
+Baugatz
+Bauernfeind
+Bauerlein
+Bau
+Batun
+Battistone
+Batteen
+Batko
+Batistich
+Bater
+Batcheller
+Batarse
+Bastow
+Bassuk
+Bassolino
+Bassel
+Bason
+Basilone
+Basich
+Bascle
+Bascetta
+Bartush
+Bartrum
+Bartlet
+Barthelmes
+Bartberger
+Bartash
+Barsoum
+Barsanti
+Barrott
+Barrom
+Barriner
+Barnhurst
+Barnell
+Barkle
+Barkes
+Barillaro
+Bargerstock
+Barganier
+Baremore
+Bardney
+Barda
+Barbot
+Barbie
+Barayuga
+Barager
+Bantz
+Bandulin
+Banasiak
+Balzarini
+Balwin
+Balton
+Balsiger
+Balmos
+Balmir
+Ballestero
+Ballek
+Balick
+Balian
+Balestra
+Balensiefen
+Balduf
+Balckburn
+Balasa
+Balafoutas
+Baksi
+Bakowski
+Baklund
+Bakko
+Bakey
+Bakanauskas
+Baj
+Baio
+Bainard
+Baima
+Baillet
+Baich
+Bahrmasel
+Bahrke
+Bahoora
+Bagsby
+Bagger
+Badena
+Badders
+Backfisch
+Bacik
+Bachler
+Bachleda
+Bachhuber
+Bachert
+Babiracki
+Baatz
+Azzarito
+Azzarella
+Azulay
+Azotea
+Azeem
+Ayoob
+Ayola
+Ayles
+Ayersman
+Ayaia
+Axthelm
+Ax
+Awtry
+Avrett
+Avilar
+Aveni
+Avellino
+Aurelia
+Aumend
+Auletta
+Augustson
+Augustave
+Aughe
+Auerswald
+Aubrecht
+Athalone
+Atanacio
+Atamian
+Astrologo
+Astrella
+Aspinall
+Asman
+Ashlin
+Ashenfelter
+Aschenbrener
+Ascheman
+Ascenzo
+Asante
+Asa
+Arvayo
+Artmann
+Artice
+Art
+Arslan
+Arrott
+Arrojo
+Arrizola
+Arriano
+Arrendell
+Arps
+Aronstein
+Aronow
+Aronica
+Arntz
+Arnst
+Arnio
+Arne
+Armengol
+Armantrout
+Arlt
+Arkadie
+Arjune
+Arismendez
+Arimas
+Aries
+Ariel
+Argandona
+Arflack
+Areola
+Arenales
+Ardman
+Arciga
+Arciba
+Archacki
+Arcaro
+Arcano
+Arbogust
+Arauz
+Aranas
+Aquil
+Aquero
+Apresa
+Appiah
+Appert
+Apostal
+Apodace
+Apadoca
+Antrobus
+Antoniuk
+Antione
+Antinarelli
+Antich
+Anslow
+Ansbro
+Annicchiarico
+Angleberger
+Angelson
+Angello
+Andruzzi
+Androsky
+Androlewicz
+Andrion
+Andringa
+Andracki
+Andra
+Ancelet
+Anastas
+Anast
+Anagnost
+Amsley
+Amsdell
+Amsberry
+Amsbaugh
+Amoruso
+Amoa
+Amici
+Amesbury
+Ambrosia
+Ambrogi
+Amack
+Alvia
+Alvaro
+Alvanas
+Altrogge
+Altomare
+Altmire
+Altenbach
+Alsheimer
+Alquisira
+Alouf
+Aloisi
+Aloe
+Almiron
+Allford
+Allex
+Allery
+Allenbach
+Allegrucci
+Alig
+Alicuben
+Alfisi
+Alferez
+Alfandre
+Alf
+Alexion
+Alevras
+Alessandrini
+Alesi
+Alescio
+Alegre
+Alea
+Aldecoa
+Alcini
+Albrittain
+Albrashi
+Alawdi
+Ala
+Aksamit
+Akima
+Akel
+Akahi
+Ajose
+Ajayi
+Aivao
+Aiu
+Ainge
+Ailshire
+Aidt
+Aicklen
+Ahuja
+Ahr
+Aholt
+Agle
+Agamao
+Affeld
+Aeschbacher
+Aeling
+Adriance
+Adkin
+Adhami
+Adeyemo
+Ades
+Adelgren
+Addicks
+Adamitis
+Ada
+Acor
+Acimovic
+Accomando
+Accola
+Acampora
+Abuaita
+Abshear
+Abrantes
+Abramovich
+Abrachinsky
+Abilay
+Abellera
+Abeles
+Abdula
+Abdon
+Abbed
+Abati
+Abascal
+Aavang
+Aadland
+Zylka
+Zwolak
+Zwingman
+Zwerschke
+Zwack
+Zurin
+Zupp
+Zumbrunnen
+Zukoski
+Zukor
+Zukas
+Zuanich
+Zoumis
+Zoulek
+Zou
+Zorra
+Zorich
+Zomorodi
+Zolty
+Zolondek
+Zolnoske
+Zoldesy
+Zoldak
+Zocklein
+Zlotnik
+Ziraldo
+Zipf
+Zinsli
+Ziniewicz
+Zindell
+Zin
+Zimmerebner
+Zimmel
+Zimm
+Zills
+Zilla
+Zilka
+Zietz
+Zietlow
+Ziemski
+Zielesch
+Zieler
+Zieglen
+Ziegenbein
+Ziegelbauer
+Ziegel
+Ziech
+Zicker
+Zicherman
+Zich
+Ziccardi
+Zgoda
+Zeschke
+Zerko
+Zerhusen
+Zepka
+Zents
+Zeni
+Zeme
+Zematis
+Zema
+Zella
+Zelkin
+Zelenski
+Zeilinger
+Zeidan
+Zegarelli
+Zeanah
+Zdon
+Zbikowski
+Zazula
+Zavesky
+Zavasky
+Zaruba
+Zarrineh
+Zarrillo
+Zarraluqui
+Zarling
+Zaring
+Zaretsky
+Zarebski
+Zanini
+Zanin
+Zangl
+Zaner
+Zand
+Zampieri
+Zaltz
+Zaloudek
+Zall
+Zalk
+Zalar
+Zakowski
+Zajc
+Zahran
+Zahnen
+Zagroba
+Zagel
+Zagara
+Zagami
+Zaffuto
+Zachmann
+Zachariades
+Zaccagnino
+Zaccagnini
+Zaborski
+Zabloudil
+Zabarkes
+Yvon
+Yusef
+Yuricic
+Yuill
+Yuenger
+Yuasa
+Ysbrand
+Yourshaw
+Younkers
+Youngdahl
+Youngblut
+Youkers
+Youkanaa
+Yorkey
+Yoneyama
+Yonamine
+Yoeckel
+Yodis
+Yocius
+Yocham
+Yobst
+Yeubanks
+Yetto
+Yerigan
+Yerbic
+Yentsch
+Yennard
+Yemchuk
+Yax
+Yaun
+Yasurek
+Yasui
+Yaskiewicz
+Yantzer
+Yantz
+Yanosky
+Yanek
+Yandle
+Yance
+Yanagi
+Yambao
+Yamakawa
+Yagoda
+Yaekel
+Yackeren
+Yacavone
+Yacano
+Ximines
+Xaimoungkhoun
+Wysock
+Wyont
+Wynott
+Wynans
+Wylde
+Wyett
+Wydner
+Wurzbacher
+Wulfing
+Wruck
+Wroe
+Wrobliski
+Wrobbel
+Wrights
+Wraspir
+Wrape
+Woytowicz
+Woy
+Worthan
+Worstel
+Worsfold
+Worrel
+Worbington
+Wools
+Woollen
+Woolems
+Woodmancy
+Woodhull
+Woodgate
+Woodfield
+Woodcox
+Woock
+Wonsik
+Wolven
+Wolslegel
+Wolny
+Wolma
+Wollyung
+Wollin
+Wolley
+Wollan
+Wolkow
+Wolke
+Wolever
+Woleslagle
+Wolansky
+Wojnicki
+Wohner
+Wohlfahrt
+Wohler
+Wloch
+Wittlin
+Wittkopp
+Wittenborn
+Wittels
+Withiam
+Withfield
+Wisz
+Wissel
+Wisseh
+Wislocki
+Wiscombe
+Wischmeyer
+Wischman
+Wirebaugh
+Winzelberg
+Winterstein
+Wintersmith
+Winterroth
+Winrich
+Winograd
+Winlock
+Winley
+Winkley
+Wings
+Winfred
+Winebaugh
+Windover
+Windly
+Winarski
+Wimbs
+Wimber
+Wiltgen
+Willmschen
+Williver
+Willinghurst
+Williamston
+Willenbrock
+Willars
+Willamson
+Wileman
+Wileczek
+Wildenberg
+Wildeman
+Wilcutt
+Wilch
+Wilby
+Wilbers
+Wikstrom
+Wigman
+Wigle
+Wigelsworth
+Wietzel
+Wiesneski
+Wienert
+Wienecke
+Wienandt
+Wieloch
+Wielgosz
+Wiedmann
+Wieckowski
+Wiece
+Wieand
+Widmar
+Widhalm
+Widgeon
+Widerski
+Widdows
+Widdop
+Widdison
+Widby
+Wida
+Whyne
+Whyel
+Whybrew
+Whittman
+Whittall
+Whitler
+Whitinger
+Whitewater
+Whitescarver
+Whitemarsh
+Whitecloud
+Whit
+Whistlehunt
+Whinnery
+Whillock
+While
+Whilby
+Wheldon
+Wheatcroft
+Whapham
+Whaite
+Wettlaufer
+Wetterer
+Wettach
+Wetsel
+Wethern
+Westrum
+Westlie
+Westgaard
+Westerhof
+Westerfeld
+Westad
+Wesly
+Wesberry
+Werring
+Werre
+Wernz
+Wermter
+Werkmeister
+Werbelow
+Wentzlaff
+Weniger
+Wengreen
+Wendolski
+Wendelberger
+Wempa
+Weltzin
+Welti
+Weltch
+Wellnitz
+Wellenstein
+Wekenmann
+Weitze
+Weitman
+Weisholz
+Weishar
+Weisbaum
+Weinraub
+Weinbauer
+Weinbach
+Weidig
+Weiderhold
+Wehrwein
+Wehrs
+Wehrly
+Wehnes
+Wehn
+Wegge
+Weerts
+Weemhoff
+Weekey
+Wedman
+Weder
+Weckman
+Weckhorst
+Weaklend
+Wauters
+Wauer
+Waud
+Wattenberg
+Watte
+Watling
+Waszkiewicz
+Wasmus
+Wasilko
+Washor
+Wartchow
+Warshauer
+Warsham
+Warrender
+Warnstaff
+Warmuth
+Warmington
+Wardrup
+Wardhaugh
+Wardall
+Warchal
+Warboys
+Wanty
+Wanous
+Wanlass
+Wangstad
+Waneka
+Wandless
+Wandel
+Wanda
+Wamser
+Wamhoff
+Walvatne
+Waltemeyer
+Walsingham
+Walljasper
+Wallet
+Wallerich
+Walkling
+Walkers
+Walezak
+Waldroff
+Waldhoff
+Waldall
+Walbright
+Walat
+Wakita
+Waka
+Waisner
+Waiki
+Waiden
+Wagle
+Wagenblast
+Wadusky
+Wadden
+Waclawski
+Wackenhut
+Wackenheim
+Wachal
+Waananen
+Waack
+Vy
+Vukcevic
+Vreugdenhil
+Vreeman
+Vrazel
+Vranes
+Vranek
+Voytek
+Voves
+Vormelker
+Vorachek
+Vontungeln
+Vonniederhaus
+Vonner
+Vonhagen
+Vondrak
+Vondielingen
+Vonasek
+Vonallmen
+Voltaire
+Vollucci
+Vollick
+Vollenweider
+Volante
+Voitier
+Vogts
+Vocu
+Voci
+Voccia
+Vliet
+Vliem
+Vizarro
+Vizard
+Vittorini
+Vitro
+Vitolas
+Vititoe
+Viteo
+Visnic
+Visher
+Visel
+Viscia
+Viscera
+Vis
+Virrueta
+Virola
+Viren
+Vinz
+Vinke
+Vinger
+Vind
+Vinagre
+Viltz
+Villwock
+Villifana
+Villiard
+Villetas
+Villasana
+Villarin
+Villante
+Villacana
+Vile
+Vilcheck
+Vilardi
+Vigueras
+Vigoren
+Vignovich
+Vignaux
+Vignarath
+Vigier
+Vieweg
+Vietti
+Vietor
+Viegas
+Viebrock
+Vidals
+Victorin
+Vicsik
+Vicic
+Vicens
+Viapiano
+Vetsch
+Vetri
+Vertiz
+Versluis
+Verrilli
+Verrelli
+Verrecchia
+Verni
+Vernetti
+Vermeer
+Verling
+Verlato
+Verkler
+Verkamp
+Verghese
+Verducci
+Verant
+Venzeio
+Venturella
+Ventress
+Venton
+Venhorst
+Venerable
+Veneman
+Ven
+Velverton
+Velunza
+Velmontes
+Vellutini
+Vellekamp
+Veleta
+Veldkamp
+Velazques
+Veino
+Veigel
+Veeneman
+Vavro
+Vauters
+Vattes
+Vaszily
+Vastakis
+Vasiloff
+Vasilauskas
+Vasconcelos
+Vars
+Varos
+Varnon
+Varkey
+Vares
+Varenhorst
+Vardy
+Varcoe
+Vanwye
+Vanwoert
+Vanwieren
+Vanvickle
+Vantreese
+Vansyckle
+Vanstrander
+Vansteenburg
+Vanstee
+Vanslander
+Vanproosdy
+Vanpoucke
+Vanpoppelen
+Vanpatton
+Vanosdel
+Vannelli
+Vanmiddleswor
+Vanloh
+Vanlith
+Vankoten
+Vanisouvong
+Vanholland
+Vanhekken
+Vanharlingen
+Vanhandel
+Vangemert
+Vaneyck
+Vanert
+Vaneps
+Vanegdom
+Vandesteene
+Vanderschaege
+Vanderkam
+Vanderheiden
+Vandergriend
+Vanderark
+Vandeputte
+Vandenbergh
+Vandegraaff
+Vandebogart
+Vandamme
+Vandalsen
+Vandagriff
+Vanclief
+Vanboven
+Vanbecelaere
+Vanartsdalen
+Vanaller
+Vanakin
+Vanabel
+Valrie
+Valrey
+Valotta
+Vallangeon
+Valladolid
+Valaitis
+Vala
+Vair
+Vaidya
+Vaid
+Vagt
+Vagle
+Uyeno
+Uson
+Us
+Urwin
+Urtado
+Ursino
+Urry
+Urquiza
+Urps
+Urmeneta
+Urlaub
+Uribazo
+Urhahn
+Ure
+Urch
+Urbanic
+Urata
+Urankar
+Ur
+Uppinghouse
+Unthank
+Unland
+Unikel
+Ungvarsky
+Ungerleider
+Ungerecht
+Underkoffler
+Umlauf
+Umbdenstock
+Ulrick
+Uliano
+Uldrich
+Ulch
+Ulberg
+Uknown
+Ukena
+Uk
+Uhri
+Uhde
+Udley
+Uboldi
+Tzeremes
+Tysor
+Tyrus
+Tyrol
+Tyl
+Tyksinski
+Tycer
+Tyberg
+Twitt
+Tweden
+Tuy
+Tuton
+Tuter
+Tustison
+Tuschhoff
+Turso
+Turrigiano
+Turowski
+Turnbo
+Turnball
+Turlich
+Turli
+Turla
+Turkin
+Turke
+Turi
+Tuong
+Tulk
+Tulip
+Tugman
+Tuggles
+Tufano
+Tucknott
+Tuccillo
+Tubeszewski
+Tuason
+Tsuzuki
+Tsunoda
+Tschannen
+Trytten
+Trybala
+Truskowski
+Trueba
+Trueax
+Truden
+Trucchi
+Trotti
+Trongone
+Tromble
+Tromblay
+Trokey
+Troiani
+Troglin
+Trodden
+Troccoli
+Tritz
+Tritch
+Trischitta
+Trisch
+Trippet
+Triplette
+Trinca
+Trimmell
+Trilling
+Trieger
+Treworgy
+Trevorrow
+Trevillion
+Trevigne
+Trevett
+Tretter
+Treston
+Trepagnier
+Trentinella
+Trenkle
+Trenh
+Trenbeath
+Tremelling
+Treider
+Treib
+Treftz
+Tredennick
+Trecroci
+Trebil
+Traves
+Traversa
+Tratar
+Traster
+Trasport
+Trank
+Trampe
+Trammer
+Trame
+Trachte
+Toyoshima
+Towley
+Tovias
+Touvell
+Tout
+Toussant
+Tourikis
+Toten
+Tosten
+Tosic
+Tosches
+Tortoriello
+Tortorice
+Torstrick
+Torset
+Torrijos
+Torrie
+Torress
+Torred
+Torra
+Torma
+Torkildsen
+Toppi
+Toporek
+Topolosky
+Topick
+Topez
+Toper
+Toncrey
+Tompsett
+Tompkin
+Tomory
+Tommolino
+Tomjack
+Tombs
+Tombrello
+Tomaszycki
+Tomaski
+Tolzmann
+Tolston
+Tolosky
+Toldness
+Tokuoka
+Tokihiro
+Tokay
+Tok
+Tojo
+Tointon
+Tohill
+Togni
+Tognazzini
+Todeschi
+Tobola
+Tobeck
+Toala
+Toadvine
+Tllo
+Tkacz
+Titchener
+Titch
+Tissot
+Tiso
+Tirri
+Tipka
+Tintle
+Tinneberg
+Tinius
+Tinelli
+Tin
+Timmreck
+Timmerberg
+Timinsky
+Timi
+Timchak
+Tillberry
+Tilgner
+Tiff
+Tieszen
+Tiemeyer
+Tiemens
+Tiell
+Tiehen
+Tidey
+Tick
+Ticas
+Tiboni
+Tiberio
+Tibbert
+Thyne
+Thurton
+Thurau
+Thune
+Thrune
+Threets
+Thorngren
+Thornbrugh
+Thorin
+Thongdy
+Thommarson
+Thoene
+Thoben
+Thoams
+Thixton
+Thistlethwait
+Thingvold
+Thiesfeld
+Thierauf
+Thielbar
+Thiebeault
+Thiara
+Thews
+Theophilus
+Theodoratos
+Thenhaus
+Theam
+Thay
+Thalmann
+Thake
+Thady
+Tevlin
+Tevebaugh
+Testen
+Tesseneer
+Tervort
+Terri
+Terrey
+Terres
+Terrasas
+Terney
+Termeer
+Terlecki
+Terheggen
+Terhark
+Terhar
+Terepka
+Terault
+Terando
+Teppo
+Tepler
+Teper
+Tent
+Tenpas
+Tennill
+Tennett
+Tenley
+Templer
+Tempe
+Temp
+Teltschik
+Telschow
+Telle
+Tekippe
+Teitsort
+Teitenberg
+Tei
+Tegarden
+Teffeteller
+Tefera
+Teesdale
+Teemer
+Teekasingh
+Teddick
+Tebay
+Tebar
+Teats
+Teano
+Teagues
+Teachman
+Teabo
+Tchakian
+Tazzara
+Tayor
+Tavorn
+Tavira
+Taverna
+Tave
+Tautuiaki
+Tatters
+Tatevosian
+Tassey
+Taschereau
+Tarzia
+Tarring
+Tarrien
+Tarras
+Tarkenton
+Tariq
+Tardio
+Tarascio
+Tara
+Tappeiner
+Tannen
+Tankersly
+Tanious
+Tangren
+Tangredi
+Tangert
+Tamulis
+Tamburrino
+Tambasco
+Tamargo
+Tamanaha
+Talluto
+Taki
+Takeshita
+Takemura
+Takaoka
+Tajiri
+Taintor
+Tahu
+Tags
+Taglieri
+Tafel
+Tadiello
+Tacket
+Taborda
+Tabolt
+Tabisola
+Tabian
+Taback
+Szymansky
+Szwejbka
+Szweda
+Szufat
+Szubinski
+Szerlong
+Szekula
+Szczygiel
+Szczepanek
+Szalay
+Szafryk
+Syrek
+Syphard
+Synan
+Symmonds
+Sydner
+Swirsky
+Swires
+Swietoniowski
+Swickheimer
+Swets
+Swetland
+Swenk
+Sweetin
+Swavely
+Swatt
+Swatsworth
+Swatski
+Swartzmiller
+Swartzbeck
+Swartzbaugh
+Swansen
+Swalley
+Swaisgood
+Swails
+Swaggert
+Svrcek
+Svinth
+Svetz
+Svetlik
+Sutulovich
+Suttell
+Susswein
+Sussex
+Susor
+Susoev
+Susich
+Susana
+Surwillo
+Suran
+Sunn
+Sunkel
+Sundling
+Sundholm
+Sumsion
+Sump
+Summar
+Sumlar
+Suminski
+Sumi
+Sumas
+Sulzman
+Sultana
+Sullinger
+Suleski
+Sulcer
+Sul
+Sukeforth
+Suing
+Suglia
+Sugiki
+Suggett
+Sueltenfuss
+Suders
+Sudar
+Suchecki
+Sucharzewski
+Suchanek
+Subler
+Suben
+Subasic
+Styborski
+Stvil
+Stumme
+Stulick
+Studyvin
+Stubson
+Stuble
+Stubits
+Stubenrauch
+Strysko
+Struggs
+Strudwick
+Strowd
+Stroub
+Stroth
+Stropko
+Stroinski
+Strnad
+Stritzke
+Stritzinger
+Strittmater
+Strieker
+Strickert
+Strength
+Stremlow
+Stremel
+Strejcek
+Streitmatter
+Streif
+Streb
+Streams
+Straws
+Strausberg
+Strathy
+Strathman
+Strater
+Straseskie
+Strapp
+Stranger
+Strande
+Stramiello
+Strakbein
+Strachn
+Stoyer
+Stoyanoff
+Stowman
+Stowbridge
+Stove
+Stoutt
+Stoutenburg
+Stouer
+Stouder
+Store
+Stoppkotte
+Stopa
+Stolts
+Stolinski
+Stolecki
+Stole
+Stojanovic
+Stofsky
+Stoffregen
+Stoffels
+Stoffa
+Stoesz
+Stodolski
+Stockett
+Stittsworth
+Stipek
+Stinett
+Stillion
+Stillinger
+Stiel
+Stiehl
+Stiegler
+Stieg
+Stickrod
+Sticht
+Stibbins
+Stevener
+Steudeman
+Stetzel
+Sterr
+Sternal
+Sterback
+Stephco
+Stenman
+Stemmerman
+Stemme
+Stemarie
+Stelting
+Stellings
+Steir
+Steinlicht
+Steiniger
+Steinbrenner
+Steidinger
+Stehney
+Stehly
+Stefka
+Steffel
+Stefanovich
+Steeno
+Steeneck
+Steenburgh
+Steckline
+Steckelberg
+Stazenski
+Stavis
+Staum
+Stauffacher
+Stauder
+Staude
+Statzer
+Stasinos
+Starwalt
+Starrs
+Starnauld
+Starek
+Stapleford
+Stapf
+Stapels
+Stansifer
+Stanojevic
+Stanick
+Standring
+Standrew
+Standke
+Standford
+Stancle
+Stanciel
+Stamnos
+Stamison
+Stallons
+Stallion
+Stallbaumer
+Stailey
+Staie
+Staiano
+Stahnke
+Stahle
+Stageman
+Stacken
+Stachecki
+Stableford
+Stabb
+Sramek
+Squines
+Spurzem
+Sprock
+Springate
+Spreng
+Spratte
+Sprang
+Sprake
+Spotwood
+Splain
+Spiwak
+Spitznogle
+Spirito
+Spirek
+Spingola
+Spincic
+Spillett
+Spika
+Spigelman
+Spielmann
+Spetter
+Sperl
+Spenard
+Speilman
+Speigel
+Speice
+Speach
+Spaugh
+Spatafore
+Spatafora
+Spar
+Spanski
+Spannaus
+Spanish
+Spanfellner
+Spalinger
+Spagnolia
+Spadea
+Spadafore
+Spadaccini
+Spachtholz
+Spach
+Spacek
+Sozzi
+Sowels
+Soulasinh
+Souffront
+Soucier
+Sotolo
+Soteros
+Sotero
+Soter
+Sossaman
+Soshnik
+Sorrick
+Soron
+Soroa
+Sornsen
+Sorgente
+Sordahl
+Sonza
+Sontheimer
+Sonstroem
+Sonoski
+Sonnenfeld
+Sonderup
+Somani
+Soman
+Somalski
+Solymani
+Solton
+Soloveichik
+Solmonson
+Sollberger
+Solkowitz
+Solimini
+Soleman
+Solders
+Soldavini
+Solanki
+Sohm
+Sodek
+Sode
+Socks
+Sockalosky
+Sochan
+Sobilo
+Soapes
+Snyders
+Snowman
+Snowdy
+Sniffin
+Snetting
+Snellman
+Snellenberger
+Snellen
+Snellbaker
+Sneathen
+Sneath
+Smyrl
+Smull
+Smolko
+Smithheart
+Smiht
+Smestad
+Sluter
+Slupe
+Slomkowski
+Slomka
+Slomba
+Sliz
+Slipp
+Slim
+Slightam
+Sleper
+Sledz
+Slechta
+Slaughterbeck
+Slaughenhoupt
+Slaight
+Sladick
+Slader
+Skye
+Skupski
+Skroch
+Skripko
+Skrine
+Skreen
+Skradski
+Skorski
+Skornik
+Skokowski
+Skok
+Skocilich
+Skinnen
+Skillington
+Skemp
+Skay
+Skattebo
+Skagerberg
+Siwik
+Sivik
+Sitar
+Sitaca
+Sission
+Sissac
+Sisney
+Siruta
+Sirmon
+Sirkoch
+Siriano
+Siracuse
+Sipler
+Sipho
+Sinkovich
+Sinkey
+Sinistore
+Singo
+Sinclaire
+Simunovich
+Simuel
+Simril
+Simpton
+Simpliciano
+Simoson
+Simonis
+Simoncini
+Simister
+Simison
+Simenez
+Simco
+Simcheck
+Silvi
+Silveri
+Silvano
+Silletto
+Sillavan
+Siles
+Silbernagel
+Sigwart
+Sigona
+Signs
+Signaigo
+Sigmond
+Sigars
+Siemek
+Siem
+Sieloff
+Sieligowski
+Siefke
+Siebeneck
+Siebenberg
+Siderman
+Siderine
+Sidberry
+Sicilia
+Sichta
+Sibrel
+Sibell
+Sibayan
+Shyu
+Shvey
+Shuter
+Shumski
+Shulund
+Shulte
+Shuker
+Shugars
+Shufford
+Shubrick
+Shub
+Shouldice
+Shotton
+Shotkoski
+Shost
+Shortsleeve
+Shorette
+Shopen
+Shont
+Shonerd
+Shone
+Shomin
+Shomer
+Sholl
+Shoger
+Shirts
+Shirota
+Shinholster
+Shindle
+Shinaberry
+Shimura
+Shimsky
+Shimo
+Shillinger
+Shilleh
+Shihadeh
+Shierling
+Shewbridge
+Shevitz
+Sheumaker
+Shettle
+Shers
+Sherren
+Shern
+Sherling
+Sherle
+Sheridon
+Sherdon
+Shelter
+Shelmon
+Shelling
+Shelko
+Sheline
+Shelhamer
+Shekey
+Shekarchi
+Sheinberg
+Shehata
+Sheffo
+Shebchuk
+Shearing
+Sheaks
+Shazier
+Shayne
+Shawnee
+Shawhan
+Shaud
+Shastri
+Sharr
+Sharlin
+Shark
+Sharits
+Sharf
+Share
+Shapskinsky
+Shape
+Shankland
+Shames
+Shalhoup
+Shaftic
+Shadiack
+Shackle
+Shabala
+Sevick
+Sevedge
+Seurer
+Sette
+Servan
+Serva
+Serrett
+Serrand
+Serisky
+Sering
+Serie
+Serianni
+Sereda
+Sequin
+Senti
+Senosk
+Senno
+Senner
+Senna
+Senerchia
+Sendro
+Sencabaugh
+Semonick
+Semetara
+Sembler
+Selvaggio
+Seltzen
+Selser
+Sellek
+Sellberg
+Selking
+Seliba
+Selfe
+Seki
+Seifarth
+Seielstad
+Sehorn
+Sehl
+Segur
+Segrave
+Sefcovic
+Seeton
+Seek
+Seecharan
+Seeberger
+Sedman
+Sedano
+Secunda
+Seburg
+Sebold
+Sebastion
+Seate
+Seashore
+Seard
+Seang
+Seaney
+Seace
+Seabert
+Sczygiel
+Scurti
+Scullen
+Scroggy
+Scripter
+Scowden
+Scorsone
+Scoleri
+Scocca
+Scire
+Sciotti
+Sciera
+Scibilia
+Sciabica
+Schwisow
+Schwier
+Schweinert
+Schweinberg
+Schweiker
+Schweigart
+Schweickert
+Schwass
+Schwarzenbach
+Schwarts
+Schwarm
+Schwamberger
+Schwalenberg
+Schwabenbauer
+Schwabauer
+Schuttler
+Schutjer
+Schuring
+Schure
+Schuppert
+Schuner
+Schulthess
+Schulteis
+Schulle
+Schuhmacher
+Schuermann
+Schuepfer
+Schuele
+Schrott
+Schrope
+Schrauder
+Schrandt
+Schouviller
+Schonert
+Schonack
+Scholzen
+Scholnick
+Schoffstall
+Schoenthal
+Schoenstein
+Schoenhut
+Schoenhard
+Schoeneman
+Schoemer
+Schoborg
+Schnicke
+Schneidtmille
+Schneiders
+Schmunk
+Schmoyer
+Schmeider
+Schmale
+Schlottman
+Schlitzer
+Schlipp
+Schlink
+Schliesser
+Schlieper
+Schlesselman
+Schlensker
+Schleis
+Schlein
+Schleck
+Schlabaugh
+Schiver
+Schirpke
+Schindel
+Schimler
+Schiltz
+Schillings
+Schiffelbein
+Schiebel
+Schiaffino
+Schettig
+Schetrompf
+Schessler
+Scherler
+Scheppe
+Schepens
+Schellman
+Schellhammer
+Scheirman
+Scheibelhut
+Schei
+Schech
+Scheaffer
+Schattner
+Schatt
+Scharte
+Schappell
+Schanding
+Schanbacher
+Schan
+Schaming
+Schamburek
+Schaeffler
+Schadle
+Schadegg
+Schabot
+Schaberg
+Schaadt
+Scerra
+Scercy
+Scattergood
+Scarset
+Scarrow
+Scarritt
+Scarpaci
+Scarles
+Scarce
+Scanlin
+Scalice
+Scali
+Scahill
+Sazama
+Saysithideth
+Sayres
+Sayavong
+Sawlivich
+Sawczyszyn
+Savo
+Savina
+Savilla
+Savela
+Savasta
+Saurel
+Saupe
+Sauberan
+Satunas
+Sattley
+Satterley
+Satiago
+Satchel
+Saska
+Sarvey
+Saroukos
+Sarnowski
+Sarnoff
+Sarli
+Sarley
+Sarelas
+Sardi
+Sarconi
+Sarbacher
+Saragusa
+Saraceno
+Sar
+Sappenfield
+Sanzotta
+Santy
+Santorella
+Santopolo
+Santin
+Santiesteban
+Santhuff
+Santell
+Sansburn
+Sanpaolo
+Sanocki
+Sannon
+Sannella
+Sanlucas
+Sanjabi
+Sangrey
+Sangi
+Sanghvi
+Sangh
+Sanfiorenzo
+Sandrowicz
+Sandoual
+Sandora
+Sandlian
+Sandi
+Sandholm
+Samuelsen
+Samu
+Sampedro
+Samorano
+Samok
+Samide
+Samber
+Samain
+Saltzgaber
+Saltonstall
+Saltern
+Salte
+Salonia
+Salmond
+Sallas
+Saliva
+Saler
+Salek
+Saldibar
+Salabarria
+Sakon
+Sakelaris
+Sake
+Sajorda
+Sajor
+Sahni
+Sagoes
+Saglimbeni
+Sagehorn
+Sagayaga
+Safdeye
+Safa
+Sadlon
+Sadbury
+Sadahiro
+Sache
+Sacavage
+Sacarello
+Sables
+Sabean
+Sabates
+Sabataso
+Saager
+Saa
+Rzucidlo
+Rzeszutko
+Ryther
+Rylant
+Ryks
+Ryherd
+Ryhal
+Rygalski
+Rybacki
+Rviz
+Ruys
+Ruuska
+Ruttman
+Ruttinger
+Ruts
+Ruter
+Rutana
+Rusten
+Russnak
+Rusinko
+Rusi
+Rushiti
+Rushia
+Rushdan
+Ruscetti
+Rusboldt
+Ruppenthal
+Rupke
+Rundahl
+Rund
+Rummer
+Rummans
+Rumler
+Ruminski
+Rumfola
+Rull
+Ruise
+Ruggle
+Ruescher
+Ruegsegger
+Ruegger
+Rudzik
+Rudney
+Rudisail
+Rudis
+Rudduck
+Rucky
+Ruckdeschel
+Rubins
+Rubenzer
+Rozo
+Rox
+Rowzee
+Rownd
+Rowey
+Rowcliffe
+Rovinsky
+Roup
+Rottner
+Rothmiller
+Rothgery
+Rothbart
+Rotenberg
+Rotando
+Roswick
+Rosu
+Rossum
+Rossetto
+Rosseter
+Rosselli
+Roskos
+Roskopf
+Rosenholm
+Rosencranz
+Rosenbrook
+Rosella
+Rosebaugh
+Rosbough
+Rosan
+Roofe
+Ronson
+Ronhaar
+Rones
+Ronchetto
+Romeno
+Rombs
+Romanoski
+Romanini
+Romanick
+Roloson
+Rollock
+Rollheiser
+Rollans
+Rold
+Rolark
+Rokisky
+Roja
+Roik
+Rohaley
+Rognstad
+Rofkahr
+Roethel
+Roessner
+Roesser
+Roehrman
+Roehrenbeck
+Roegge
+Roefaro
+Rody
+Rodrigo
+Rodricks
+Rodino
+Rodillas
+Rodia
+Rodenbaugh
+Rodell
+Rodeiguez
+Rodarta
+Rockenbach
+Robley
+Robes
+Robertello
+Robello
+Robella
+Robak
+Roarx
+Rivlin
+Rivira
+Rivena
+Ritzert
+Ritell
+Ritcheson
+Riska
+Risberg
+Ripke
+Rinkel
+Riniker
+Ringman
+Ringlein
+Ringelheim
+Ringbloom
+Rinde
+Rincones
+Rimson
+Rimar
+Riliford
+Rihn
+Rihanek
+Rigoni
+Riggott
+Riffon
+Rievley
+Rieve
+Riesenweber
+Rieg
+Rieff
+Riedell
+Riechers
+Rieber
+Rieben
+Riebeling
+Ridpath
+Ridler
+Riddock
+Rickson
+Rickmon
+Rickley
+Rickie
+Richrdson
+Ribot
+Riblet
+Rhyme
+Rhoney
+Rhed
+Rhead
+Rezek
+Reynvaan
+Reynoza
+Reye
+Rexwinkle
+Revord
+Reven
+Reveal
+Reutlinger
+Reuland
+Reuer
+Retzler
+Rettke
+Retterbush
+Retort
+Reth
+Resureccion
+Restifo
+Resnikoff
+Rerko
+Repsher
+Repress
+Reppell
+Repinski
+Repenning
+Renze
+Rennix
+Renning
+Renney
+Rennell
+Renfer
+Rener
+Rendino
+Renaker
+Remmen
+Rementer
+Remenaric
+Relkin
+Reiterman
+Reist
+Reisser
+Reisling
+Reisert
+Reise
+Reio
+Reinmiller
+Reine
+Reill
+Reigner
+Reifler
+Reifel
+Reidenbach
+Rehnquist
+Rehler
+Rehfield
+Rehfeldt
+Rehberger
+Regler
+Regel
+Regehr
+Refsell
+Reen
+Reem
+Reeher
+Reech
+Reeber
+Redstone
+Redo
+Redish
+Redhage
+Redenz
+Redell
+Reddrick
+Redder
+Reckley
+Reckleben
+Recine
+Rebusi
+Rebuldela
+Rebera
+Rebell
+Rebeles
+Reavley
+Reau
+Reatherford
+Reaney
+Reaid
+Reagans
+Reado
+Razinger
+Razey
+Raza
+Rayside
+Raymos
+Raygosa
+Rawding
+Raw
+Ravens
+Ravenhorst
+Rav
+Rauzman
+Rautenberg
+Rausin
+Rauner
+Raudebaugh
+Rattner
+Ratleff
+Rathmell
+Rathgeb
+Ratermann
+Rataczak
+Rasher
+Rashdi
+Rashada
+Rasbery
+Rarang
+Rapose
+Rapa
+Ransick
+Ranos
+Rankhorn
+Raniero
+Rang
+Randzin
+Rancher
+Rances
+Rancatti
+Ramoutar
+Ramnarase
+Ramlakhan
+Ramiro
+Ramiriz
+Ramez
+Rameriez
+Rambus
+Ramaswamy
+Ramagos
+Ramadanovic
+Ramadan
+Ralko
+Ralat
+Rakel
+Raju
+Rajtar
+Raja
+Rairdon
+Raimo
+Raif
+Raiche
+Raheja
+Raheem
+Rahall
+Raguso
+Rafanan
+Rafalko
+Raes
+Radzavich
+Radune
+Radulescu
+Raduenz
+Radsek
+Radom
+Radell
+Rackett
+Racilis
+Rachi
+Rach
+Racedo
+Rabold
+Rabner
+Rabern
+Rabenstein
+Rabelo
+Quintas
+Quinlisk
+Quine
+Quincey
+Quilantang
+Quicksey
+Quereto
+Quelette
+Quaresma
+Quann
+Quall
+Quails
+Quaas
+Qadir
+Pytlovany
+Pybus
+Putaski
+Purwin
+Purter
+Purple
+Purol
+Purkiss
+Pummel
+Pults
+Pultorak
+Pullian
+Puller
+Pulham
+Puletasi
+Puidokas
+Puhuyaoma
+Puffinburger
+Puesey
+Puelo
+Puddephatt
+Pucillo
+Puc
+Przepiora
+Prys
+Pruzansky
+Pruyn
+Prust
+Prusinski
+Prus
+Pruette
+Provis
+Provine
+Proue
+Protz
+Prosonic
+Prophett
+Pronto
+Pronovost
+Proksch
+Prok
+Proietto
+Proia
+Proenza
+Probus
+Prizzi
+Privalsky
+Prisock
+Printy
+Primozich
+Priefert
+Pridham
+Preus
+Prettner
+Prester
+Pressel
+Preskar
+Premer
+Premeaux
+Preisinger
+Preisendorf
+Prehm
+Pregeant
+Preedom
+Pralle
+Prag
+Pradel
+Prabhakar
+Poyser
+Poupard
+Potterson
+Pottebaum
+Potolsky
+Poto
+Potes
+Postlethwaite
+Postin
+Pospishil
+Poskus
+Posik
+Portsche
+Portolese
+Porrini
+Poro
+Porietis
+Poppenhagen
+Poppen
+Poppel
+Pontonio
+Ponting
+Pono
+Pomposo
+Pomponio
+Pomplun
+Pomo
+Pomeranz
+Pomella
+Pomberg
+Pomares
+Polucha
+Polselli
+Polnau
+Pollins
+Pollara
+Polisky
+Polio
+Policz
+Policar
+Polchinski
+Polashek
+Polakowski
+Polaco
+Poitevin
+Poister
+Pointon
+Poinson
+Poinsett
+Pogar
+Poetter
+Podmore
+Poczobut
+Pockette
+Pocasangre
+Pobre
+Plys
+Plunket
+Plumpton
+Pluemer
+Plover
+Ploetz
+Ploense
+Plocek
+Plikerd
+Pleet
+Pleasure
+Plazza
+Plaxico
+Platko
+Platania
+Plassmann
+Plantier
+Plantenga
+Plancarte
+Plakke
+Pladson
+Pizzano
+Pivin
+Pittsinger
+Pittmann
+Pitsenbarger
+Pitonyak
+Pitmon
+Pitfield
+Pitek
+Pitassi
+Pistulka
+Pistole
+Piske
+Pishko
+Pisegna
+Pirnie
+Pirkey
+Pippitt
+Piorkowski
+Pinna
+Pinkton
+Pinks
+Pinkerman
+Pinchbeck
+Pimpare
+Pilloud
+Pillitteri
+Pilakowski
+Pikus
+Pikula
+Pikkarainen
+Pijanowski
+Pigao
+Piette
+Pietrzykowski
+Pietryga
+Pietropaolo
+Pies
+Piersaul
+Pieri
+Piepenbrink
+Pieloch
+Pieffer
+Picucci
+Pickl
+Pickhardt
+Picini
+Picerni
+Picaro
+Piatak
+Pianalto
+Piacquadio
+Phoun
+Phonharath
+Phomsoukha
+Phommaseng
+Phinazee
+Phillippy
+Phillians
+Philavong
+Phernetton
+Pheonix
+Phenes
+Pfotenhauer
+Pfleiderer
+Pfleider
+Pflanz
+Pfieffer
+Pfeiff
+Pfautz
+Pezzica
+Pevez
+Pevehouse
+Petrunger
+Petrullo
+Petrucco
+Petrson
+Petrilla
+Petrides
+Petrauskas
+Petkus
+Petiet
+Petgrave
+Peterschick
+Petaway
+Pesner
+Pesiri
+Pesin
+Pesa
+Pervine
+Pertubal
+Perschall
+Perrucci
+Perow
+Peroddy
+Perocho
+Perno
+Perloff
+Peria
+Pergerson
+Pereyda
+Pereria
+Pereiro
+Perdzock
+Perchinski
+Peraro
+Peques
+Pepito
+Pentek
+Pentaris
+Pennison
+Pennewell
+Pennacchio
+Penington
+Peninger
+Pengelly
+Penegar
+Pencek
+Penale
+Penaherrera
+Pembrook
+Pelyo
+Pelligra
+Pele
+Pekala
+Peine
+Peightal
+Peers
+Peerbolt
+Pedaci
+Ped
+Pectol
+Pecot
+Pecos
+Pecorelli
+Pechart
+Pebbles
+Peatry
+Pearle
+Peard
+Peakes
+Peaches
+Paywa
+Paysinger
+Payes
+Pawelczyk
+Pavoni
+Pavlovic
+Pavelec
+Pavan
+Paullus
+Pauldo
+Patuto
+Patruno
+Patoine
+Patock
+Patka
+Pata
+Pastiva
+Pastick
+Passwater
+Passineau
+Passi
+Pasquino
+Pasquel
+Pasquarelli
+Pason
+Paskert
+Pashley
+Pashia
+Partis
+Partido
+Parsi
+Parrill
+Parolari
+Parisio
+Pariser
+Parents
+Parduhn
+Parden
+Parcel
+Parbo
+Paray
+Papson
+Pappa
+Papillion
+Papik
+Paparella
+Papai
+Paoletto
+Pantone
+Pannhoff
+Pankowski
+Pangelina
+Pangallo
+Panda
+Panciera
+Panchana
+Panasci
+Panarella
+Paltanavage
+Palsgrove
+Palovick
+Paloma
+Palmiotto
+Palmiero
+Palmerton
+Palmerin
+Pallet
+Pallesen
+Pallazzo
+Palitti
+Palischak
+Paliotta
+Palifka
+Palenik
+Palecek
+Palczewski
+Palasik
+Palacious
+Pala
+Pahnke
+Pahls
+Paguirigan
+Pagnozzi
+Pagliarini
+Paduano
+Paddison
+Padavano
+Pacubas
+Packingham
+Packebush
+Pacius
+Paci
+Pacey
+Pacas
+Pac
+Ozolins
+Ozog
+Ozminkowski
+Oyuela
+Owston
+Ovsanik
+Overlie
+Overbo
+Oven
+Ovard
+Ourso
+Ouderkirk
+Ottis
+Otterholt
+Otomo
+Otley
+Osuch
+Ostling
+Ostlie
+Ostheimer
+Osterstuck
+Osterdyk
+Ostenson
+Osten
+Ossowski
+Osso
+Osmon
+Osle
+Oskins
+Osendorf
+Osburne
+Osawa
+Ortic
+Ortenzio
+Orrantia
+Orrala
+Orouke
+Orone
+Orofino
+Orkwis
+Orizetti
+Oris
+Orines
+Orgovan
+Orgain
+Orendorff
+Orendain
+Oree
+Orea
+Ordner
+Ordas
+Orbeck
+Oravec
+Opray
+Ophus
+Opela
+Opatrny
+Opara
+Oosterhof
+Onusko
+Onstead
+Onorata
+Onitsuka
+Onishea
+Oneel
+Ondrusek
+Omundson
+Omoyosi
+Omdahl
+Oltz
+Olton
+Olrich
+Olquin
+Olp
+Olmscheid
+Olm
+Olivio
+Oliverson
+Oliven
+Olis
+Oline
+Olexa
+Olesnevich
+Olesky
+Oleksiak
+Oldani
+Olcus
+Oksen
+Okolo
+Okojie
+Okerblom
+Okajima
+Ohrenich
+Ohms
+Ohmann
+Ohland
+Oguinn
+Ogiba
+Ogeen
+Oge
+Oganyan
+Offenbacker
+Oesterreich
+Oerther
+Oelschlager
+Odore
+Odonal
+Odonahue
+Odiase
+Odenwald
+Odens
+Odear
+Octave
+Ockey
+Ochwat
+Ochotorena
+Ochiltree
+Och
+Ocejo
+Ocano
+Obstfeld
+Obleness
+Obiesie
+Oberloh
+Oberfell
+Obannion
+Oakleaf
+Oak
+Nyswonger
+Nyseth
+Ny
+Nuvallie
+Nusom
+Nush
+Nurnberger
+Nunziata
+Nunev
+Nudelman
+Nucklos
+Nuce
+Novik
+Noury
+Notik
+Notari
+Nosis
+Nosel
+Northcraft
+Northcote
+Norskog
+Norrid
+Norquest
+Normann
+Norma
+Norlund
+Norley
+Norcott
+Norbeck
+Noonon
+Nooney
+Nonaka
+Nollora
+Nollman
+Nolda
+Nolau
+Nol
+Nogueras
+Nogowski
+Nogosek
+Noftsger
+Noeldner
+Nocum
+Nocket
+Nocar
+Noaks
+Niverson
+Nittinger
+Nitterhouse
+Nitkowski
+Niten
+Nitchals
+Nissila
+Nishiguchi
+Nippert
+Nippe
+Ninos
+Nine
+Nimocks
+Nimmer
+Nilsby
+Nill
+Nikolas
+Nikirk
+Niimi
+Nii
+Niheu
+Nihei
+Nigg
+Niforos
+Niezgoda
+Nieva
+Niethamer
+Niesman
+Nienow
+Niedermayer
+Niedecken
+Nied
+Niebyl
+Nie
+Nicotera
+Nicolet
+Nicolaisen
+Nickolls
+Nickol
+Nickleson
+Nickelston
+Nichois
+Nicewarner
+Niceswander
+Nicarry
+Nicar
+Nhep
+Ngueyn
+Nguen
+Ngov
+Nghe
+Newsted
+Newnum
+Newer
+Newburg
+Newall
+Nevland
+Neugin
+Neuenfeldt
+Neuby
+Nestel
+Nesseth
+Nervis
+Nerpio
+Nenninger
+Nemzek
+Nemoede
+Nemer
+Nelmark
+Nellem
+Neithercutt
+Neiswander
+Neisius
+Neish
+Neihart
+Neiderhiser
+Nehmer
+Negrisor
+Negrette
+Nefzger
+Neeper
+Neelon
+Needels
+Needam
+Nealley
+Nealen
+Nealeigh
+Nayee
+Nawn
+Navone
+Navejas
+Navedo
+Navar
+Naud
+Natiello
+Nathoo
+Nasson
+Naselli
+Nase
+Naschke
+Narez
+Nares
+Nappier
+Napoletano
+Napihaa
+Naone
+Nannini
+Nannie
+Nania
+Nanda
+Nampel
+Nalepka
+Najjar
+Nahass
+Naeve
+Naecker
+Nadell
+Myrum
+Myint
+Myhr
+Myerscough
+Muterspaw
+Mutana
+Muszar
+Mustafaa
+Must
+Mussenden
+Mussen
+Mushett
+Musetti
+Musemeche
+Musel
+Muscaro
+Murrock
+Murrie
+Murrain
+Murilla
+Murelli
+Murayama
+Murai
+Munzell
+Munteanu
+Munt
+Munshower
+Munlin
+Muni
+Munding
+Munda
+Mulvehill
+Mulry
+Mulliner
+Mullice
+Mullaly
+Muhr
+Muhn
+Mugica
+Muether
+Muehlberger
+Muehlbach
+Muccia
+Mrowka
+Mrotz
+Mrochek
+Mracek
+Moznett
+Moyse
+Moxham
+Mowris
+Moutoux
+Moussette
+Mousley
+Moun
+Moulinos
+Mostrom
+Mostert
+Mosses
+Moskovitz
+Mosinski
+Mosgrove
+Mosebach
+Moschetto
+Morway
+Morthland
+Morta
+Morsbach
+Morreau
+Morowski
+Moroles
+Morlas
+Morgenstein
+Morasch
+Moranda
+Moralis
+Moraitis
+Moraites
+Moote
+Moorcroft
+Montier
+Montie
+Montesa
+Monteros
+Montefusco
+Montecalvo
+Montazami
+Montaya
+Monsky
+Monsegur
+Monnet
+Monjaras
+Moniot
+Monholland
+Monet
+Monestine
+Monds
+Mondry
+Mondo
+Mondino
+Momsen
+Momaya
+Molski
+Mollins
+Molitoris
+Mokbel
+Moistner
+Moilien
+Mohring
+Mohrbacher
+Mogro
+Moerman
+Moellman
+Modero
+Moczo
+Mocco
+Mocarski
+Mobus
+Mizukami
+Miyares
+Miyahara
+Miyagishima
+Mittendorf
+Mittelstadt
+Mitsakos
+Mith
+Mita
+Misura
+Missler
+Misrahi
+Misnick
+Misemer
+Miscovich
+Miscavage
+Misasi
+Mirich
+Miravalle
+Miras
+Miramon
+Mioduszewski
+Mio
+Minster
+Minnier
+Minneweather
+Minnehan
+Minkel
+Miners
+Mineah
+Mincher
+Minatra
+Minato
+Minari
+Minardo
+Milush
+Miltner
+Milster
+Milovich
+Milman
+Millraney
+Millot
+Millisor
+Milliren
+Millimaki
+Millich
+Milland
+Milkovich
+Militano
+Mileti
+Milek
+Mildren
+Milder
+Milch
+Milbert
+Milbauer
+Milanowski
+Milanese
+Mikulecky
+Mikulak
+Mikita
+Mikelsen
+Mihlfeld
+Mihatsch
+Mihalkovic
+Mihalko
+Mignogna
+Migl
+Miessner
+Mieras
+Midcap
+Mickleberry
+Michocki
+Michelman
+Michales
+Michalenko
+Mias
+Mhoon
+Mezza
+Mezquita
+Mezera
+Meyette
+Meyerhoffer
+Meyerhofer
+Meury
+Meuller
+Mettle
+Metter
+Mettee
+Metta
+Metroka
+Metevier
+Metaxas
+Mestrovich
+Messa
+Mesidor
+Meschino
+Meryman
+Merrett
+Merrbach
+Merone
+Merkling
+Merickel
+Mercante
+Meo
+Mensinger
+Menist
+Menino
+Menhennett
+Mengarelli
+Menez
+Menesez
+Mendelowitz
+Mencl
+Men
+Mellors
+Mellom
+Mellencamp
+Mellekas
+Melkonian
+Melish
+Meleski
+Melero
+Melchin
+Melbert
+Melandez
+Melander
+Meisels
+Meighen
+Mehtala
+Mehserle
+Meholick
+Mehalic
+Megna
+Meginnis
+Meggitt
+Meggers
+Meger
+Meeter
+Meeske
+Meeder
+Medows
+Mednick
+Medich
+Mediate
+Median
+Medez
+Medbery
+Medak
+Mebus
+Meason
+Meanor
+Meager
+Mcwethy
+Mcvean
+Mcthune
+Mcsweeny
+Mcspedon
+Mcsharry
+Mcravin
+Mcraven
+Mcquistion
+Mcquilkin
+Mcquaide
+Mcquage
+Mcpherren
+Mcpeck
+Mcnaney
+Mcmindes
+Mcmilliam
+Mcmenomy
+Mcmarlin
+Mcmahill
+Mcloy
+Mcloone
+Mclear
+Mclaughlan
+Mckoan
+Mckerley
+Mckerchie
+Mckeone
+Mckennie
+Mckellan
+Mckaig
+Mcinally
+Mchendry
+Mcgwier
+Mcguirt
+Mcgugin
+Mcgready
+Mcgraff
+Mcgrade
+Mcgorry
+Mcglothian
+Mcglory
+Mcgavisk
+Mcgarrigle
+Mcever
+Mcelmurry
+Mcelheny
+Mcelhattan
+Mcdaries
+Mcdargh
+Mccumiskey
+Mccredie
+Mccraven
+Mccoyle
+Mccoppin
+Mccombie
+Mccloughan
+Mccleve
+Mcclenty
+Mcclennan
+Mcclees
+Mccleer
+Mcclearen
+Mccaskin
+Mccartin
+Mccamy
+Mccammack
+Mccaman
+Mccalop
+Mccaffity
+Mcburrows
+Mcburrough
+Mcbrady
+Mcalphin
+Mcalhaney
+Mcaboy
+Mazikowski
+Mazar
+Mayzes
+Maymon
+Mayeski
+Maycumber
+Mayala
+Maxin
+Maute
+Mauss
+Mauritz
+Maurey
+Maulin
+Matuszeski
+Matusik
+Matuseski
+Mattu
+Mattier
+Matthys
+Matteucci
+Matsuhara
+Matsen
+Matrejek
+Matlick
+Mathewes
+Mathal
+Matey
+Matesic
+Materna
+Matelic
+Matarese
+Matalavage
+Mataalii
+Mastrocovi
+Mastrobuono
+Mastoris
+Mastera
+Mastenbrook
+Mastella
+Massaglia
+Maslyn
+Masley
+Masin
+Masiclat
+Mashiah
+Mashek
+Mascot
+Maschke
+Maschio
+Masch
+Marzinske
+Marxen
+Marville
+Marushia
+Marungo
+Maruffo
+Maruca
+Martinz
+Martinetto
+Martinetti
+Martinea
+Martincic
+Martig
+Marske
+Marshalsea
+Marsette
+Marroguin
+Marreo
+Marquena
+Marona
+Marola
+Marmie
+Markstrom
+Marksbury
+Markrof
+Markovitz
+Markevich
+Markette
+Marius
+Maritt
+Marionneaux
+Marinos
+Marinese
+Maricich
+Marhoefer
+Margiotta
+Maren
+Marecki
+Marcone
+Marcoline
+Marcolina
+Marchuk
+Marcelynas
+Marcaida
+Marbus
+Marazzi
+Marazas
+Marashio
+Maranville
+Marani
+Marandi
+Marander
+Marade
+Mapalo
+Manza
+Manylath
+Manvelyan
+Manusyants
+Mantuano
+Mantsch
+Mantell
+Mantano
+Mansmann
+Manship
+Manozca
+Mannie
+Mannes
+Manliguis
+Manigold
+Maniatis
+Mania
+Mangon
+Manginelli
+Mangicavallo
+Mangiaracina
+Mangas
+Mangaoang
+Manford
+Mandiola
+Manchini
+Mamoran
+Mammucari
+Mamer
+Malys
+Malvin
+Malvaez
+Malusky
+Maltie
+Maltbie
+Malphurs
+Malotte
+Malloch
+Malkasian
+Malit
+Malis
+Malinski
+Malinchalk
+Malicote
+Malich
+Maletz
+Malesky
+Maler
+Malekzadeh
+Maleh
+Malech
+Malbaurn
+Malara
+Malakan
+Malakai
+Malafronte
+Malady
+Makley
+Makekau
+Majmundar
+Majersky
+Maiten
+Mainiero
+Mainello
+Mailes
+Maigret
+Mahusay
+Maharg
+Mahany
+Maguet
+Magowan
+Magone
+Magnall
+Magleby
+Maglaya
+Maginn
+Magin
+Magil
+Maggs
+Maggie
+Magelssen
+Magaw
+Magario
+Magallanez
+Maeweather
+Madura
+Madrueno
+Madinger
+Madho
+Maderas
+Maddry
+Madaris
+Maczko
+Macugay
+Macrowski
+Macomb
+Macnab
+Maclaurin
+Maclauchlan
+Mackynen
+Macksoud
+Macks
+Mackney
+Mackintosh
+Mackinder
+Maciej
+Macie
+Machowski
+Machol
+Machinsky
+Machalek
+Macchione
+Macall
+Macafee
+Mabus
+Mabins
+Mabane
+Maassen
+Lysen
+Lynaugh
+Lykens
+Luvian
+Luttenegger
+Lutkins
+Lutchman
+Lutao
+Luskin
+Luskey
+Lungren
+Lundburg
+Lumm
+Lulic
+Lulewicz
+Lukaszewicz
+Luiso
+Luhnow
+Lugg
+Lugardo
+Lufsey
+Luetmer
+Luepke
+Ludtke
+Luczkowiak
+Luckhardt
+Luckenbaugh
+Lucken
+Luchenbill
+Lubke
+Lubell
+Lube
+Lubbock
+Lozon
+Loze
+Lozaya
+Loynd
+Loxley
+Lowthorp
+Lowek
+Loviska
+Lovig
+Lovgren
+Loverink
+Lovensheimer
+Lounsbery
+Loukota
+Loughnan
+Loughborough
+Loudenslager
+Lotson
+Lothspeich
+Lotan
+Lossa
+Losolla
+Losier
+Lorna
+Lorimor
+Lori
+Lorett
+Lorens
+Loreg
+Loreaux
+Lorandeau
+Loque
+Lopus
+Lopriore
+Lootens
+Lookadoo
+Lonneman
+Lonn
+Longiotti
+Longhini
+Longendyke
+Longbotham
+Londre
+Londagin
+Lonabaugh
+Lomu
+Lominy
+Lomboy
+Lomartire
+Lollie
+Lokker
+Loia
+Loi
+Logrono
+Logosso
+Loggains
+Loflen
+Lofink
+Lofgreen
+Loewenthal
+Loeurm
+Loerzel
+Loeppke
+Loepp
+Loegering
+Lodholz
+Lockey
+Lockbaum
+Lochte
+Lochan
+Lobur
+Loban
+Llorca
+Lloid
+Llewlyn
+Llanez
+Liwanag
+Livernoche
+Litzenberg
+Litano
+Lissard
+Lisko
+Liscio
+Lipskar
+Lipscombe
+Lipschutz
+Lipphardt
+Lipinsky
+Lipani
+Lions
+Linnertz
+Links
+Linkowski
+Linko
+Lingafelter
+Lingafelt
+Lindzy
+Lindman
+Lindert
+Lindersmith
+Linders
+Linderholm
+Lindburg
+Lindaman
+Lincicome
+Linberg
+Linamen
+Limke
+Lilyquist
+Liloia
+Lillpop
+Lillick
+Lillich
+Lilien
+Lighter
+Liggin
+Lifton
+Lifsey
+Lifford
+Lifer
+Liest
+Liem
+Lidke
+Liddiard
+Lick
+Lichtenwalner
+Lichtenfeld
+Lichak
+Licerio
+Licausi
+Licause
+Libman
+Libera
+Liaw
+Leya
+Lewitt
+Lewandoski
+Levoy
+Levitin
+Leviston
+Leventer
+Levenhagen
+Leveillee
+Leve
+Lettre
+Letsche
+Lesiak
+Leshinsky
+Leriche
+Leri
+Lepri
+Leppke
+Lepping
+Lepp
+Lepo
+Leonhard
+Leonello
+Leona
+Leofsky
+Lensing
+Lenoci
+Lennington
+Lennihan
+Lenn
+Lenkiewicz
+Lenis
+Lenertz
+Lenehan
+Lenci
+Lenarz
+Lemucchi
+Lemick
+Lelah
+Lelacheur
+Lejenne
+Leitman
+Leithoff
+Leistiko
+Leipert
+Leibert
+Leibe
+Lehnertz
+Leheny
+Lehar
+Lehane
+Legorreta
+Legoff
+Legleu
+Legions
+Leggat
+Leggans
+Legaard
+Left
+Leesmann
+Leemaster
+Leemans
+Ledwig
+Ledlie
+Lederhos
+Lecorchick
+Leclear
+Leclare
+Leckman
+Leckbee
+Lebrecque
+Lebahn
+Leavenworth
+Leatherberry
+Leamer
+Leady
+Lazzeri
+Lazarini
+Lazarine
+Laza
+Layng
+Lawshe
+Lawman
+Lawer
+Laware
+Lavista
+Lavis
+Laviola
+Lavinder
+Lavern
+Lavene
+Lavelett
+Lavanway
+Lavanchy
+Lavalette
+Lavala
+Lavadie
+Lava
+Lautzenheiser
+Lautt
+Lauser
+Laurimore
+Lauridsen
+Laurey
+Laurenti
+Laurente
+Laurenitis
+Laurelli
+Laukitis
+Laud
+Lattrell
+Lattner
+Latterell
+Latten
+Lattari
+Lattanzi
+Latif
+Lastufka
+Lasswell
+Lasseson
+Lassa
+Laslo
+Laski
+Lashute
+Lashmet
+Larrieu
+Larrier
+Larribeau
+Laronda
+Larney
+Larita
+Lariccia
+Largin
+Larez
+Lardin
+Larch
+Lapusnak
+Laprete
+Lapre
+Lapradd
+Lapore
+Lapinsky
+Lapid
+Laperriere
+Laos
+Lantto
+Lantaff
+Lanson
+Lanois
+Lanius
+Lanini
+Languirand
+Languell
+Langstraat
+Langreck
+Langkabel
+Langill
+Langeness
+Langefels
+Langarica
+Langager
+Lanfranco
+Lanfear
+Lanfair
+Landvatter
+Landolfi
+Landborg
+Lanagan
+Lampson
+Lampshire
+Lamoreux
+Lambrukos
+Lambrakis
+Lamborne
+Lambing
+Lamax
+Lamarch
+Lallave
+Lalka
+Lais
+Lairy
+Laiben
+Lahren
+Lahn
+Lahmers
+Lah
+Lagory
+Laforrest
+Laflore
+Lafkas
+Lafield
+Lafay
+Laduc
+Laderer
+Ladell
+Ladakakos
+Lacoy
+Lacki
+Lacio
+Lacinski
+Lachowsky
+Lacerda
+Lace
+Lacasa
+Labruzzo
+Labre
+Labove
+Laberpool
+Labbadia
+Labarba
+Labady
+Kytle
+Kym
+Ky
+Kwasnicki
+Kwapniewski
+Kwang
+Kuzminski
+Kuzel
+Kuwahara
+Kut
+Kusko
+Kusick
+Kuruvilla
+Kurtulus
+Kurtis
+Kurtich
+Kurkowski
+Kurkeyerian
+Kuritz
+Kurelko
+Kurcaba
+Kuralt
+Kuprewicz
+Kupetz
+Kuntzman
+Kunishige
+Kundtz
+Kulwicki
+Kulow
+Kulis
+Kuhlmey
+Kufel
+Kues
+Kuehnel
+Kudrick
+Kudlacik
+Kudej
+Kuchel
+Kuchan
+Kucha
+Kuboushek
+Kubishta
+Kubilus
+Kubert
+Kubeika
+Kubasik
+Kuakini
+Krzyston
+Krzeczkowski
+Kryzak
+Krygier
+Kry
+Krupski
+Krupke
+Krupansky
+Krumvieda
+Krumholz
+Krumbholz
+Krudop
+Krstic
+Krovious
+Krommes
+Kromm
+Krolak
+Kroes
+Kroening
+Kroener
+Kritter
+Kristy
+Krisman
+Kriege
+Kridel
+Kreul
+Kretsinger
+Kretlow
+Kresal
+Krejsa
+Kreines
+Kreig
+Krefft
+Krauskopf
+Kratt
+Krassow
+Krasnecky
+Krance
+Krajcik
+Krail
+Kraham
+Krack
+Kozloff
+Kozlak
+Kozera
+Kozee
+Koyama
+Kowalowski
+Kowalchuk
+Kovalovsky
+Kovalcheck
+Koutz
+Kotts
+Kostyk
+Kosty
+Kostohryz
+Kostiuk
+Kostis
+Kostick
+Kosofsky
+Kosman
+Kosin
+Kosier
+Kosen
+Kosco
+Koschnitzki
+Kosbab
+Kosack
+Korzep
+Korvin
+Kortkamp
+Kornrumpf
+Korfhage
+Kordus
+Korchnak
+Koppinger
+Kopinski
+Kopald
+Kooyman
+Koopmans
+Koonz
+Kooker
+Kooch
+Konzal
+Konye
+Kontogiannis
+Konruff
+Konowal
+Konopnicki
+Konopacky
+Konopacki
+Konig
+Konicki
+Konecni
+Kondel
+Konakowitz
+Komlos
+Kombe
+Komatz
+Kolm
+Kollmeyer
+Kollasch
+Kolin
+Kolden
+Kolbo
+Kolata
+Kolaga
+Kokocinski
+Koko
+Koinzan
+Kohrman
+Kohnz
+Kogler
+Koets
+Koerwitz
+Koep
+Koenecke
+Koehly
+Kockler
+Kocka
+Kociolek
+Kobie
+Knudsuig
+Knoten
+Knotek
+Knole
+Knochel
+Knobbe
+Knightstep
+Knigge
+Knife
+Kniess
+Knickelbein
+Kneisler
+Kneedler
+Knedler
+Knall
+Knable
+Klym
+Klussmann
+Kluever
+Kludt
+Klouda
+Klotzbach
+Klosowski
+Klockars
+Klinker
+Klingshirn
+Klingelhoets
+Klingelhoefer
+Klena
+Klempa
+Klemisch
+Klemens
+Klemencic
+Klemen
+Kleinhenz
+Klecha
+Klebanow
+Klebanoff
+Klave
+Klang
+Klammer
+Klamet
+Klaers
+Klacic
+Kjar
+Kivisto
+Kivel
+Kitzrow
+Kitzerow
+Kitz
+Kiszka
+Kistenmacher
+Kisicki
+Kisak
+Kirylo
+Kirson
+Kirschke
+Kirmer
+Kirakosyan
+Kinton
+Kint
+Kinsland
+Kinlock
+Kini
+Kingsolver
+Kingdon
+Kindschuh
+Kindlimann
+Kindl
+Kindberg
+Kinas
+Kinaj
+Kimberl
+Killoy
+Killette
+Killer
+Killary
+Kilgor
+Kildoo
+Kilborne
+Kilbert
+Kil
+Kijek
+Kiewiet
+Kiever
+Kiesz
+Kiessling
+Kielar
+Kiehn
+Khosravi
+Kholodivker
+Kho
+Khatib
+Khatcherian
+Keyworth
+Keylor
+Kewanwytewa
+Kettman
+Kettlewell
+Kettl
+Kettelle
+Kethcart
+Ketay
+Keslar
+Kesby
+Kerne
+Kerk
+Kercy
+Kerchal
+Kerbel
+Kenrick
+Kennis
+Kennin
+Kennemuth
+Kennelty
+Kenkel
+Kemmerling
+Kemfort
+Kelstrom
+Kellow
+Kellom
+Kelk
+Keliiholokai
+Kelcourse
+Kekua
+Keiger
+Keglovic
+Keesecker
+Keehne
+Keedah
+Keding
+Keavney
+Keanu
+Keagy
+Keaffaber
+Keadle
+Kazemi
+Kazanowski
+Kazanjian
+Kazan
+Kawelo
+Kavanah
+Kautzer
+Kaukola
+Kaufusi
+Kauffeld
+Katowicz
+Katos
+Katheder
+Kately
+Kata
+Kastor
+Kastl
+Kassouf
+Kassler
+Kassam
+Kaskey
+Kasimis
+Kasdon
+Kaschmitter
+Kaschel
+Karratti
+Karpinen
+Karpen
+Karmann
+Karlovich
+Karlen
+Karkut
+Karin
+Kariger
+Karaffa
+Kapsos
+Kapps
+Kapnick
+Kanoa
+Kanney
+Kannas
+Kanduth
+Kampman
+Kamimura
+Kamens
+Kamemoto
+Kalvaitis
+Kaltenhauser
+Kalloch
+Kaller
+Kallenberg
+Kaliszuk
+Kalinoski
+Kalinger
+Kalich
+Kalfus
+Kalfayan
+Kalert
+Kalenkoski
+Kalen
+Kaleiwahea
+Kaleel
+Kaldas
+Kalawe
+Kalathas
+Kakos
+Kaiserman
+Kais
+Kailiponi
+Kaighn
+Kahuhu
+Kahoun
+Kahen
+Kahaleua
+Kah
+Kagy
+Kager
+Kagarise
+Kaffka
+Kaempfer
+Kaemmerer
+Kaelker
+Kady
+Kadner
+Kadlubowski
+Kadakia
+Kacynski
+Kacic
+Kach
+Kabrick
+Justman
+Justine
+Jurina
+Jurik
+Jurcik
+Junius
+Jumalon
+Julca
+Jui
+Jugan
+Juart
+Jove
+Journeay
+Joung
+Jou
+Josilowsky
+Josephsen
+Josephpauline
+Jorde
+Joor
+Jonte
+Jolie
+Johnke
+Johanningmeie
+Joerg
+Jochems
+Jilk
+Ji
+Jhonston
+Jez
+Jethva
+Jethro
+Jest
+Jesko
+Jerrel
+Jerich
+Jentsch
+Jensvold
+Jennrich
+Jenious
+Jenck
+Jemenez
+Jelle
+Jelinski
+Jeleniewski
+Jelen
+Jeffrie
+Jefford
+Jedik
+Jebbett
+Jayes
+Javarone
+Jauss
+Jaus
+Jaskolski
+Jasionowski
+Jasin
+Jarzynka
+Jarva
+Jaruis
+Jaross
+Jaret
+Jaquess
+Janovich
+Jannusch
+Jann
+Jankins
+Janitz
+Janicke
+Jangula
+Jamon
+Jammer
+Jamie
+Jameel
+Jakupcak
+Jakubczak
+Jakowich
+Jakeman
+Jagneaux
+Jagher
+Jaekel
+Jadin
+Jacobowitz
+Jackstadt
+Jackowiak
+Jackiewicz
+Jackels
+Jabour
+Izsak
+Izarraras
+Iwasa
+Iwanyszyn
+Iulo
+Iuliucci
+Iturbide
+Itkin
+Isby
+Isam
+Isales
+Isackson
+Irizarri
+Iribarren
+Irani
+Iracheta
+Iott
+Ioli
+Iodice
+Ioannidis
+Intriago
+Interrante
+Intermill
+Insco
+Inloes
+Ingrim
+Inglin
+Inglese
+Ingala
+Infield
+Inestroza
+Ineson
+Indest
+Incorvaia
+Inacio
+Imparato
+Imm
+Imfeld
+Imaizumi
+Illescas
+Ikuta
+Iino
+Ignasiak
+Igler
+Igel
+Iffert
+Idris
+Idema
+Ichinotsubo
+Ichinose
+Iburg
+Iarossi
+Iannaccone
+Iams
+Iacovissi
+Hytros
+Hyten
+Hysinger
+Hylle
+Hylinski
+Hvizdos
+Huyghe
+Huus
+Hutsler
+Hutchen
+Hustus
+Huso
+Husni
+Huslander
+Huska
+Hush
+Huschle
+Husayko
+Husanini
+Hurtis
+Hurter
+Hurrington
+Hurrigan
+Hurl
+Hurban
+Hunten
+Hundemer
+Humerickhouse
+Humbel
+Hulstine
+Hulm
+Huitzacua
+Hughlett
+Huger
+Huewe
+Huels
+Hudrick
+Hudek
+Huckeby
+Hubright
+Hubric
+Hubel
+Hsi
+Hryniewich
+Hrovat
+Hronick
+Hribar
+Hozempa
+Hoxworth
+Howryla
+Howison
+Howieson
+Howdeshell
+Hoving
+Hovi
+Hovelson
+Hovell
+Houten
+Housten
+Housekeeper
+Houpe
+Houp
+Houman
+Houghland
+Hougas
+Hothan
+Hotchkin
+Hoste
+Hosie
+Hosendove
+Hoseman
+Hoseck
+Hoschouer
+Horwood
+Horuath
+Hortillosa
+Horth
+Horsfield
+Horniak
+Hornby
+Hormander
+Horii
+Hores
+Horaney
+Horal
+Hopskins
+Hoppesch
+Hoopengardner
+Hoomana
+Hoolihan
+Hoof
+Honzel
+Honse
+Honohan
+Hongo
+Hongerholt
+Homola
+Homerding
+Homchick
+Holy
+Holvey
+Holsing
+Holshue
+Hollenberg
+Hollemon
+Holla
+Holka
+Holifeild
+Holets
+Holdt
+Holdness
+Holdiness
+Holda
+Holcey
+Holbein
+Hoium
+Hoisl
+Hohstadt
+Hohowski
+Hoh
+Hogy
+Hogsten
+Hogsette
+Hoggins
+Hofler
+Hoffstot
+Hoffschneider
+Hoffee
+Hoevel
+Hoernemann
+Hoeper
+Hoener
+Hoene
+Hoeke
+Hoeg
+Hoeflich
+Hoeffner
+Hoeffliger
+Hoecker
+Hoeck
+Hoe
+Hodgen
+Hodan
+Hockema
+Hochschild
+Hobkirk
+Hnatow
+Hledik
+Hjalmarson
+Hitzler
+Hittman
+Hisman
+Hirstein
+Hirschhorn
+Hirsche
+Hirkaler
+Hiraoka
+Hiraki
+Hipwell
+Hippo
+Hinsey
+Hinkey
+Hinish
+Hingst
+Hingle
+Hindin
+Hinahon
+Himelstein
+Hillburg
+Hillaire
+Hilgert
+Hildred
+Hildahl
+Hilcher
+Higueros
+Higle
+Higinbotham
+Hieserich
+Hidvegi
+Hidrogo
+Hickton
+Hickonbottom
+Hickert
+Hibl
+Heyveld
+Heydel
+Hevner
+Hevesy
+Heverley
+Heverin
+Heusley
+Heuberger
+Hettwer
+Hett
+Heter
+Hesters
+Hessong
+Hessing
+Hessenthaler
+Hessell
+Hessee
+Hesby
+Herzberger
+Herwood
+Herting
+Herscher
+Herschel
+Herrling
+Herrig
+Herriage
+Herrel
+Herre
+Herpolsheimer
+Hernanders
+Hermosura
+Hermie
+Hermens
+Herklotz
+Herkert
+Herby
+Herbster
+Herbison
+Herbers
+Herbein
+Heppeard
+Henrick
+Henrey
+Henretta
+Henneberg
+Hennagin
+Henington
+Henifin
+Heney
+Henesey
+Henehan
+Hendy
+Henderosn
+Hender
+Hendee
+Henby
+Henaire
+Hemrich
+Hemmie
+Hemmes
+Hemlepp
+Heminover
+Hemauer
+Helvy
+Helsing
+Helmy
+Helmstetler
+Helmink
+Helmcamp
+Hellar
+Hellams
+Helker
+Helgesen
+Helfritz
+Helena
+Hele
+Hektner
+Hejl
+Heitschmidt
+Heitger
+Heinzmann
+Heinzen
+Heininger
+Heineken
+Heimrich
+Heimbaugh
+Heiermann
+Hehr
+Hegre
+Hegmann
+Hefler
+Hefflinger
+Heese
+Heeney
+Heemstra
+Hedrich
+Hedgespeth
+Hedemann
+Hedegore
+Heddlesten
+Heckenberg
+Hebig
+Hebden
+Hebda
+Heatly
+Heathershaw
+Hearson
+Heally
+Healan
+Heads
+Hazleton
+Hazarika
+Hayhoe
+Haydal
+Hayburn
+Hawthrone
+Hawman
+Hawkey
+Hawf
+Havice
+Havercroft
+Hautamaki
+Hauskins
+Haulter
+Haugrud
+Hauan
+Hatzenbuhler
+Hatzenbuehler
+Hattub
+Hattier
+Hatteyer
+Hatstat
+Hathway
+Hataway
+Hassick
+Hassian
+Hasselman
+Hasselbarth
+Hasper
+Haspel
+Haske
+Hasgill
+Hasen
+Harviston
+Harvilla
+Harvilicz
+Harver
+Hartzer
+Hartup
+Hartsough
+Hartsch
+Hartly
+Hartlep
+Hartlein
+Hartkopf
+Harthun
+Hartfiel
+Hartery
+Hartert
+Hartage
+Harsey
+Harrey
+Harrett
+Harral
+Haroutunian
+Harmeyer
+Harlowe
+Harloff
+Hardyman
+Hards
+Hardrict
+Hardmon
+Hardigree
+Hardenburg
+Hardell
+Hardebeck
+Hardaman
+Hardaker
+Harcey
+Harbick
+Harajli
+Happer
+Hapgood
+Hanstein
+Hansbury
+Hanold
+Hanohano
+Hano
+Hanns
+Hannifan
+Hannes
+Hanko
+Hanis
+Hanenkrat
+Hanemann
+Hanek
+Handzel
+Handwerker
+Handwerk
+Handsaker
+Handrick
+Handelsman
+Handal
+Hancin
+Hanbury
+Hanaway
+Hanahan
+Hams
+Hammerly
+Hammeren
+Hammatt
+Hammarlund
+Hamling
+Hamiss
+Hamiel
+Hamelinck
+Hambrecht
+Halo
+Hallinger
+Hallick
+Halifax
+Halgrimson
+Halfmann
+Halder
+Hald
+Halburnt
+Halberstam
+Halaby
+Haker
+Haken
+Haine
+Hagos
+Hagmaier
+Hagenson
+Hagene
+Hagenbrok
+Hagenbaugh
+Hafter
+Haffling
+Haeger
+Haegele
+Hade
+Hadder
+Hadcock
+Haczynski
+Hackle
+Hachigian
+Hachez
+Habrock
+Habowski
+Habina
+Haberkamp
+Habben
+Habash
+Haaby
+Gyatso
+Gwalthney
+Guziec
+Guziak
+Guys
+Guynup
+Gutzwiller
+Guttmann
+Gutting
+Gutteridge
+Guterrez
+Guszak
+Gusky
+Gusciora
+Gurry
+Gurrieri
+Guritz
+Gunst
+Gundry
+Gundert
+Gulsvig
+Gulisano
+Gulinson
+Guittar
+Guitard
+Guisti
+Guiski
+Guinto
+Guinther
+Guinnip
+Guilliam
+Guillerault
+Guilfoil
+Guijarro
+Guidetti
+Guiberteau
+Guger
+Guevera
+Guetersloh
+Guerini
+Guella
+Guedea
+Guecho
+Gudis
+Guckin
+Guberman
+Guardipee
+Guanio
+Guagliardo
+Grzegorek
+Grybel
+Grunst
+Grunlien
+Grundmeier
+Grundhoefer
+Grun
+Grumer
+Grum
+Gruhn
+Gruger
+Grudt
+Growney
+Grotts
+Groton
+Grotelueschen
+Grotberg
+Grosswiler
+Gronowski
+Gronosky
+Gronewald
+Gronert
+Groholski
+Groetken
+Groeschel
+Groene
+Grodecki
+Groceman
+Griswell
+Griseta
+Grinkley
+Grinie
+Grinberg
+Grimmius
+Grieme
+Greytak
+Grett
+Grenke
+Grenda
+Greinke
+Greeves
+Greever
+Greet
+Greenlun
+Greenler
+Greenham
+Grebin
+Grboyan
+Grawburg
+Grattelo
+Grassham
+Granvold
+Granthan
+Gransky
+Grandolfo
+Grandmaison
+Grandchild
+Granbois
+Gramolini
+Grammatica
+Gramc
+Grajek
+Grahe
+Gragson
+Gragert
+Grage
+Grafenstein
+Graetz
+Gracely
+Graceffo
+Grabarczyk
+Gouzalez
+Gouse
+Gourdin
+Goudelock
+Goud
+Gottlob
+Gottke
+Gotthelf
+Gotthard
+Gotter
+Gotsche
+Gotschall
+Gosz
+Goston
+Gossack
+Gosdin
+Gorz
+Gorrill
+Gornto
+Gornie
+Gorenberg
+Gorelli
+Gordinier
+Gora
+Gopin
+Gopie
+Goolman
+Goolden
+Goodsite
+Goodmanson
+Goodly
+Goodkin
+Goodiel
+Gonzolas
+Gonsior
+Gonseth
+Gonez
+Gonchoff
+Gonales
+Gomzales
+Gomora
+Golly
+Gollihar
+Gollhofer
+Golka
+Golinski
+Golen
+Golembeski
+Golemba
+Goldwater
+Goldstock
+Goldklang
+Goldbeck
+Golda
+Gojmerac
+Goich
+Gohlke
+Goger
+Gogel
+Goga
+Gofton
+Goffe
+Goetting
+Goeser
+Goerner
+Goerke
+Goerdel
+Goeppner
+Godsman
+Godert
+Godel
+Gobeli
+Gnas
+Glucksman
+Glotzbecker
+Gloeckner
+Glockner
+Glish
+Glickson
+Glicken
+Glew
+Glessing
+Gleichman
+Glazener
+Glave
+Glausier
+Glatzel
+Glassett
+Glasbrenner
+Gladu
+Glab
+Glaab
+Giza
+Gittler
+Gittleman
+Gittinger
+Gitting
+Gitthens
+Gissel
+Gischer
+Girst
+Girsch
+Girona
+Girillo
+Gire
+Gira
+Giovanetti
+Gionest
+Gingles
+Gingery
+Ging
+Gillstrap
+Gillson
+Gillotti
+Gillmor
+Gilliss
+Gillig
+Gillert
+Gillcrest
+Gilgour
+Gilgore
+Gilding
+Gilderman
+Gilcreast
+Gieseman
+Gieselman
+Gieringer
+Gick
+Giangrosso
+Giangregorio
+Giambra
+Giambattista
+Ghibaudy
+Ghianni
+Ghelfi
+Ghaziani
+Ghantt
+Ghant
+Ghaemmaghami
+Gey
+Getler
+Getchius
+Gesualdo
+Gesmondi
+Gerweck
+Gerwe
+Gerula
+Gertsen
+Gershey
+Gershen
+Gers
+Gerritsen
+Gerdsen
+Gerczak
+Gerbatz
+Gerba
+Gerache
+Georgl
+Georgiadis
+Georgelis
+Georgalas
+Genualdo
+Gentery
+Gennock
+Gennett
+Genett
+Gendernalik
+Genas
+Gena
+Gemmen
+Gelston
+Gellman
+Gelfo
+Gelen
+Gelbowitz
+Geibig
+Gehlhausen
+Geffre
+Geesaman
+Geel
+Gedman
+Geckles
+Gebbie
+Gearwar
+Gearlds
+Gayne
+Gayfield
+Gawlas
+Gauwain
+Gaufin
+Gauani
+Gastley
+Gastello
+Gassoway
+Gasparino
+Gaskey
+Gaser
+Gascot
+Garuti
+Garrington
+Garreh
+Garnand
+Garlits
+Garity
+Garitty
+Gariety
+Garia
+Gari
+Garetson
+Garelik
+Garding
+Garb
+Garasha
+Ganzer
+Gantert
+Ganotisi
+Ganner
+Ganison
+Ganie
+Gangell
+Gangel
+Ganesh
+Gandrud
+Ganas
+Gamby
+Gambles
+Galyan
+Galuski
+Galper
+Gallwas
+Galluzzi
+Gallups
+Gallosa
+Gallipeau
+Gallet
+Gallerani
+Gallegly
+Gallaty
+Gallaspy
+Gallander
+Galioto
+Galicinao
+Galer
+Galdon
+Galardi
+Galamay
+Galabeas
+Gala
+Gaitor
+Gagg
+Gagan
+Gaerlan
+Gadley
+Gacke
+Gacia
+Gach
+Gabrelcik
+Gabay
+Gabard
+Fylnn
+Fydenkevez
+Futter
+Fuse
+Fuscaldo
+Furstenberg
+Furmanik
+Furlone
+Furia
+Furer
+Furci
+Furbish
+Funt
+Fulker
+Fukano
+Fujino
+Fuhrmeister
+Fugo
+Fuerman
+Frymyer
+Fryling
+Frontz
+Froncek
+Fronce
+Frolich
+Froio
+Froid
+Froehle
+Frischman
+Friou
+Friot
+Frieze
+Friesz
+Friemering
+Frieman
+Friedrick
+Friedle
+Frickson
+Frickel
+Frichette
+Fricano
+Fribley
+Frewing
+Frever
+Freudenstein
+Frerking
+Frenger
+Freisner
+Fregeau
+Freedle
+Frease
+Frazey
+Frascone
+Franzmann
+Franzetti
+Frankforter
+Francy
+Franckowiak
+Francies
+Franchette
+Fralin
+Fraleigh
+Fraint
+Fragozo
+Fracchia
+Frabizzio
+Fousek
+Fouraker
+Foucault
+Fosson
+Fossati
+Fosnough
+Forts
+Forthman
+Forsting
+Forstedt
+Forshay
+Forshaw
+Forsha
+Forro
+Forno
+Forlivio
+Forkosh
+Forkan
+Forcello
+Foradori
+Fontane
+Fonger
+Foney
+Fondy
+Fondow
+Folta
+Follin
+Folliard
+Folley
+Folken
+Foiles
+Fohn
+Foggs
+Foesch
+Foertsch
+Foecking
+Fodness
+Foat
+Flot
+Flosi
+Florenz
+Florens
+Florencio
+Florea
+Florczak
+Flodin
+Flocke
+Flo
+Flentroy
+Flenard
+Fleisner
+Flecther
+Flaks
+Flagstad
+Flagel
+Fjetland
+Fixico
+Fiume
+Fitterer
+Fisette
+Firlit
+Firestein
+Fiotodimitrak
+Fioto
+Finner
+Finnefrock
+Fingado
+Finely
+Fincel
+Finau
+Fimbrez
+Filoteo
+Fillpot
+Fillare
+Filipski
+Filippo
+Filipovic
+Filipelli
+Filimaua
+Filhiol
+Filgo
+Fileds
+Filbert
+Figuera
+Figliola
+Figart
+Fietsam
+Fieselman
+Fiene
+Fieldhouse
+Fiebig
+Fidel
+Fida
+Fickert
+Fiato
+Fevold
+Feuerborn
+Fetchko
+Fesh
+Feser
+Ferruso
+Ferriolo
+Ferriola
+Ferrence
+Ferrar
+Ferran
+Ferraiz
+Feroz
+Ferone
+Fernstrom
+Fernstaedt
+Fernow
+Ferkovich
+Fergen
+Ferdolage
+Ferdinandsen
+Ferbrache
+Fennewald
+Fenk
+Fenix
+Fendler
+Fenchel
+Felske
+Fellinger
+Felicetti
+Feldpausch
+Feighan
+Feichter
+Fehrle
+Fehringer
+Fegaro
+Feener
+Feeler
+Fedorchak
+Federowicz
+Fedd
+Feauto
+Feagen
+Feaganes
+Fazzina
+Fazzi
+Faykosh
+Fayard
+Favuzza
+Favolise
+Fausset
+Fauske
+Fausel
+Fauscett
+Faulknen
+Faulkenburg
+Fatica
+Fastlaben
+Fastic
+Farzan
+Farstvedt
+Farin
+Farguharson
+Fargnoli
+Farfalla
+Farese
+Farer
+Faraldo
+Faraj
+Fara
+Fanzo
+Fanton
+Fanney
+Fanizzi
+Fanion
+Fanelle
+Falterman
+Falsetti
+Fallone
+Falkiewicz
+Falconio
+Fake
+Fairleigh
+Fahringer
+Fahrenkrug
+Faerber
+Fadley
+Fadeley
+Facundo
+Fack
+Face
+Faby
+Fabrizius
+Fabozzi
+Fabiszewski
+Fabin
+Ezpeleta
+Ezparza
+Eyrich
+Eyerman
+Ewoldt
+Ewards
+Evasco
+Evanich
+Evangelo
+Eustace
+Eugley
+Euertz
+Etulain
+Etchells
+Esson
+Esskew
+Essery
+Esselink
+Espinol
+Espenoza
+Espelien
+Espeland
+Espadas
+Esler
+Eske
+Eska
+Escuriex
+Escovar
+Escort
+Eschrich
+Eschette
+Eschen
+Eschbaugh
+Escalon
+Escalero
+Esbrandt
+Esary
+Ertman
+Eroh
+Ernesto
+Erlenbusch
+Erle
+Erke
+Erichsen
+Eric
+Erholm
+Erbstein
+Erbst
+Eppolito
+Eppihimer
+Eppich
+Entin
+Enslinger
+Enslen
+Enockson
+Ennenga
+Enman
+Englett
+Engleson
+Englerth
+Engl
+Engholm
+Engelken
+Engelkemier
+Engelhaupt
+Engelbach
+Endries
+Endow
+Endito
+Enderby
+Encallado
+Emziah
+Embt
+Embs
+Embelton
+Emard
+Elwonger
+Elvsaas
+Elumbaugh
+Elstner
+Elsmore
+Elskamp
+Elshant
+Elmblad
+Ellson
+Ellias
+Elletson
+Ellestad
+Ellert
+Ellermann
+Ellerbrock
+Elleman
+Ellars
+Elland
+Eliezrie
+Eldib
+Eldert
+Elbe
+Ekwall
+Ekholm
+Eken
+Eitnier
+Eitniear
+Eisenzimmer
+Eisenstadt
+Eisensmith
+Eiselman
+Eisbach
+Eisaman
+Eiken
+Eibell
+Ehrke
+Ehrismann
+Ehrenfeld
+Ehlman
+Egizi
+Egitto
+Eggeman
+Effron
+Ednie
+Edelbrock
+Edde
+Edd
+Economos
+Eckols
+Eckloff
+Echegoyen
+Ebia
+Eberlin
+Ebbers
+Easterbrook
+Earney
+Earleywine
+Eanni
+Eadens
+Dyron
+Dykhoff
+Dyers
+Dyda
+Dybala
+Dwane
+Dwaileebe
+Duverne
+Duve
+Dusen
+Dusatko
+Dusablon
+Durrette
+Durphey
+Durnin
+Durkes
+Durette
+Durdy
+Durch
+Duracher
+Dupray
+Dupoux
+Duponte
+Duperclay
+Dupass
+Dupar
+Dunwiddie
+Dunsing
+Dunnaville
+Duncomb
+Duncklee
+Dunay
+Dunakin
+Dumpe
+Dumes
+Dumdei
+Dumay
+Dulkis
+Dukich
+Dukas
+Duin
+Dugo
+Duewall
+Duemmel
+Duelm
+Dueber
+Dudman
+Dudak
+Duckhorn
+Duchscherer
+Ducat
+Ducas
+Dubyk
+Dubill
+Dubiansky
+Dubaldi
+Dua
+Dspain
+Drzazgowski
+Drymon
+Drylie
+Druvenga
+Druschel
+Drungo
+Droze
+Drouse
+Drott
+Drosick
+Droneburg
+Droessler
+Droesch
+Drobny
+Drizin
+Dripps
+Drinkley
+Drillock
+Driesbach
+Dretzka
+Dresner
+Drentlaw
+Drenon
+Drehs
+Drehobl
+Drda
+Draxler
+Drath
+Drapeaux
+Dragula
+Drafts
+Draft
+Dozer
+Doxtater
+Doxie
+Dowst
+Dowson
+Downton
+Dowlen
+Dowey
+Dowery
+Douty
+Doughtry
+Doughtery
+Dotzler
+Dotterer
+Dothard
+Dosher
+Dosal
+Dorso
+Dorsette
+Doro
+Dornfeld
+Dorkin
+Dorka
+Dorge
+Dorchy
+Dorame
+Dopler
+Dopico
+Doore
+Dooms
+Donnie
+Donnelley
+Donnel
+Donayre
+Donatello
+Donachie
+Dominiguez
+Domingos
+Dominga
+Dominey
+Domenget
+Dolores
+Dollyhigh
+Dollen
+Dollak
+Doleac
+Dolch
+Dolbeare
+Dokka
+Dokes
+Doire
+Doing
+Dohring
+Dohogne
+Dohnal
+Dohan
+Doerle
+Doerhoff
+Doemelt
+Doehring
+Doegg
+Dodsworth
+Dodoo
+Dodier
+Dockendorf
+Docken
+Dobrowski
+Dobrin
+Dobine
+Doberstein
+Dizer
+Dixey
+Divita
+Diven
+Divalerio
+Dituri
+Ditton
+Disspain
+Disparte
+Dismore
+Disilvestro
+Dishong
+Dishian
+Diseth
+Discenza
+Dirkson
+Dirkse
+Dirker
+Dirk
+Dipippo
+Dipinto
+Dipierro
+Dinnocenzo
+Dinizio
+Dinis
+Dingivan
+Dingfelder
+Dincher
+Dimucci
+Dimpson
+Dimpfl
+Dimitrov
+Dimarzo
+Dils
+Dilisio
+Diliberto
+Diliberti
+Diles
+Dileonardo
+Dilena
+Dijulio
+Diiulio
+Digiuseppe
+Diga
+Difillippo
+Difebbo
+Dieng
+Diekman
+Didyk
+Didriksen
+Dickus
+Dickow
+Dickeson
+Dicastro
+Dibenedetti
+Dhaliwal
+Dezenzo
+Dewyse
+Dewinter
+Dewaters
+Dewaele
+Devoto
+Devor
+Devoogd
+Deviva
+Devitis
+Devit
+Deveyra
+Devericks
+Devenuto
+Deveja
+Devaughan
+Deutschendorf
+Deuink
+Deubner
+Detzler
+Detullio
+Detore
+Dethlefsen
+Dethlefs
+Detamble
+Desrevisseau
+Desotel
+Deso
+Desmeules
+Desmaris
+Desilvio
+Deshpande
+Deschambault
+Descamps
+Desatnik
+Desamito
+Desalle
+Desak
+Derwin
+Derting
+Derrah
+Deroven
+Derosso
+Deromer
+Dermott
+Deringer
+Derico
+Derga
+Derflinger
+Derezinski
+Derck
+Derbacher
+Deranick
+Depuydt
+Depung
+Depree
+Deppert
+Depierre
+Dephillips
+Deojay
+Denzin
+Denten
+Dentel
+Dennies
+Denina
+Denger
+Deneke
+Denegre
+Denboer
+Denapoli
+Demsky
+Demsey
+Demotta
+Demmons
+Demman
+Demendonca
+Demeester
+Dembowski
+Demarce
+Deman
+Demallie
+Demaire
+Delwiche
+Delphia
+Delore
+Dellenbaugh
+Dellbringge
+Dellaratta
+Dellaporta
+Dellapenna
+Dellacioppa
+Deliberto
+Delibertis
+Delgenio
+Delcueto
+Delaurie
+Delauder
+Delatrinidad
+Delash
+Delaet
+Del
+Dekrey
+Dejoie
+Deiters
+Deimund
+Degrenier
+Degre
+Degrand
+Degon
+Degeston
+Degelbeck
+Degaust
+Degasparre
+Defreece
+Defenderfer
+Defee
+Deeken
+Dedon
+Dedinas
+Dedicke
+Dedic
+Decristofaro
+Decoud
+Decos
+Deconti
+Deckers
+Decio
+Decenzo
+Debroux
+Debrot
+Debray
+Deboef
+Debiasio
+Debettignies
+Debenedittis
+Debbins
+Debaecke
+Dearson
+Dearo
+Deardon
+Deaquino
+Deacetis
+Dayne
+Dayem
+Dax
+Dawoud
+Davitt
+Davito
+Davidoff
+Dauterman
+Daughterty
+Daugaard
+Daudelin
+Daubendiek
+Dattilio
+Datcher
+Dasovich
+Daso
+Dasilua
+Dashem
+Darou
+Darke
+Dargin
+Darga
+Darco
+Darcey
+Dapas
+Dantos
+Danson
+Danny
+Danielian
+Danchetz
+Danby
+Damrow
+Damours
+Damboise
+Dambakly
+Dambach
+Damasco
+Damann
+Dallmeyer
+Dallesandro
+Dalfonso
+Dakins
+Dakes
+Daire
+Dahill
+Daguio
+Dagis
+Dabdoub
+Czerkies
+Czarnota
+Czachor
+Czach
+Cypress
+Cynthia
+Cylkowski
+Cyfers
+Cwiakala
+Cvetkovic
+Cuzman
+Cuzick
+Cuttler
+Cutt
+Cuti
+Cutforth
+Cutchins
+Cutchall
+Cushwa
+Curo
+Curbeam
+Cunnick
+Cuneio
+Cundick
+Cumbaa
+Cultice
+Cullity
+Cullip
+Cullifer
+Cucvas
+Cuculich
+Cucino
+Cubeta
+Cser
+Crupper
+Crunkilton
+Cruden
+Crover
+Crouter
+Crough
+Crouchet
+Crosthwaite
+Croon
+Cronshaw
+Cronenberg
+Crome
+Croman
+Crognale
+Crogan
+Croasmun
+Cristofori
+Cristiano
+Crisan
+Cringle
+Crincoli
+Crill
+Crieghton
+Cridge
+Criblez
+Crellin
+Cregeen
+Creeks
+Creath
+Creacy
+Crazier
+Crawmer
+Crawhorn
+Cratin
+Crapser
+Crapse
+Cranmore
+Cramm
+Cramblit
+Cramblet
+Cragin
+Cracas
+Cozzone
+Coyco
+Coxey
+Cowper
+Cowett
+Covone
+Covill
+Coverton
+Councilman
+Coultrap
+Coulas
+Coughenour
+Cough
+Cotty
+Cotherman
+Cother
+Costantini
+Cossell
+Cossano
+Cosley
+Coslett
+Coskey
+Cosgray
+Corza
+Corvi
+Corvan
+Corsetti
+Corscadden
+Corsa
+Corrow
+Corrice
+Correro
+Correale
+Corre
+Corna
+Corke
+Corid
+Corelli
+Cordonnier
+Cordona
+Corak
+Coppler
+Copelan
+Coore
+Coonradt
+Coones
+Cookus
+Conveniencia
+Contrerras
+Contrenas
+Contorno
+Constantini
+Constantineau
+Consolver
+Conrath
+Connet
+Connerly
+Conliffe
+Conforto
+Conda
+Conca
+Conales
+Compono
+Compau
+Commendatore
+Comings
+Comboy
+Combass
+Coltrin
+Colpetzer
+Colonel
+Colombini
+Cologie
+Colla
+Colbeth
+Colbaugh
+Colasuonno
+Colapinto
+Colamarino
+Colaluca
+Colaianni
+Colafrancesco
+Colace
+Colabella
+Coggsdale
+Coffill
+Codispoti
+Codell
+Cocoros
+Cocopoti
+Cocola
+Cockley
+Cockey
+Cochron
+Coch
+Cobden
+Coatsworth
+Coarsey
+Coar
+Clymore
+Clumpner
+Clougher
+Clolinger
+Clinkingbeard
+Clineman
+Clewes
+Clemments
+Claypole
+Clayburg
+Claybron
+Claybon
+Claughton
+Clase
+Clarenbach
+Clankscales
+Clampett
+Claessens
+Claburn
+Citrin
+Cisney
+Cirri
+Cipro
+Cipkowski
+Cione
+Cinquanti
+Cink
+Cimiano
+Ciervo
+Ciers
+Cicora
+Ciciora
+Cicione
+Cicerelli
+Ciccolini
+Ciccarone
+Cicarella
+Ciarletta
+Ciaccio
+Chuta
+Chustz
+Churan
+Chumbler
+Chuba
+Chruch
+Christler
+Christinsen
+Christinat
+Christello
+Chrispin
+Chrismer
+Chrislip
+Chrisjohn
+Chrestman
+Choute
+Chough
+Chorlton
+Chomka
+Chmelicek
+Chiulli
+Chislom
+Chiras
+Chinzi
+Chinnery
+Chinick
+Chim
+Chilvers
+Chilo
+Chiarmonte
+Chiarenza
+Chiapetti
+Chhuon
+Chhour
+Chheang
+Chetram
+Chessher
+Cherrier
+Cherepy
+Cherenfant
+Chenot
+Cheli
+Checa
+Cheathan
+Chears
+Chauvaux
+Chaudoin
+Chauarria
+Chatters
+Chatlos
+Chatley
+Chasey
+Charves
+Charsky
+Charania
+Chaplen
+Chaple
+Channer
+Chander
+Champey
+Champeau
+Challen
+Chall
+Chalkley
+Chalet
+Chalcraft
+Chaix
+Chadick
+Chadbourn
+Chaban
+Cesari
+Cervoni
+Cervin
+Certalich
+Cerni
+Cerney
+Cereo
+Cerce
+Ceravolo
+Ceparano
+Centrella
+Centner
+Centano
+Cenat
+Celmer
+Celenza
+Celadon
+Cefaratti
+Cefalo
+Cedillos
+Cecilia
+Cechini
+Cecala
+Cease
+Cearns
+Cazeau
+Cayson
+Cayanan
+Cavallario
+Cauthron
+Cattrell
+Catterson
+Catrone
+Catone
+Catoggio
+Caterino
+Catching
+Catalani
+Castrataro
+Castoe
+Castles
+Castillanos
+Castellonese
+Castelhano
+Cassman
+Cassius
+Cassisse
+Cassem
+Cassani
+Cassandra
+Casola
+Caselli
+Cascone
+Casburn
+Casbeer
+Casbarro
+Carrin
+Carreker
+Carrea
+Carre
+Carrauza
+Carranzo
+Carpinello
+Carolin
+Carmolli
+Carmena
+Carmell
+Carmain
+Carlye
+Carlsten
+Carlough
+Carlone
+Caringi
+Carine
+Carin
+Carela
+Cardono
+Cardle
+Cardinali
+Cardi
+Cardera
+Carback
+Capuzzi
+Capracotta
+Cappo
+Cappleman
+Capparelli
+Caponera
+Caplener
+Capanna
+Caoili
+Caoile
+Canzio
+Cantoran
+Cantillo
+Canta
+Canonica
+Cannington
+Canniff
+Cangas
+Canevazzi
+Canes
+Caneles
+Candido
+Canders
+Cance
+Canaway
+Canarte
+Canario
+Canan
+Camren
+Campusano
+Campman
+Camm
+Caminos
+Camferdam
+Camerena
+Camell
+Camak
+Camaj
+Calway
+Calvino
+Calvetti
+Calvani
+Caltabiano
+Calnimptewa
+Calnick
+Calnen
+Calmese
+Callander
+Callabrass
+Caliz
+Calija
+Calger
+Calendine
+Calderara
+Calcara
+Calamity
+Cailler
+Caho
+Caguimbal
+Cadoff
+Caddick
+Cadavieco
+Cabos
+Cabiltes
+Cabibbo
+Cabellero
+Cabasso
+Caballes
+Cabading
+Caal
+Byra
+Byod
+Bynon
+Byner
+Bynam
+Byker
+Buzzi
+Buzzeo
+Butzen
+Buttz
+Butteris
+Butkiewicz
+Buteaux
+Bustad
+Bussone
+Busman
+Bushmaker
+Busche
+Burwinkel
+Burum
+Burtless
+Bursi
+Burrup
+Burross
+Burries
+Burrichter
+Burrelli
+Buron
+Buro
+Burnstein
+Burnaugh
+Burnap
+Burkdoll
+Buris
+Burington
+Burgun
+Burgie
+Burghard
+Burgh
+Burgas
+Burgardt
+Burga
+Burdess
+Burcin
+Burchfiel
+Burchess
+Burandt
+Buonanno
+Buonamici
+Buntjer
+Bungert
+Bundschuh
+Bumps
+Buman
+Bulosan
+Bullocks
+Bullie
+Bularz
+Buland
+Bujarski
+Buhmann
+Buhman
+Bugna
+Buglisi
+Buggy
+Buemi
+Budke
+Buder
+Budds
+Buddie
+Buczak
+Buckwald
+Buckovitch
+Buckholtz
+Buckhanan
+Buchetto
+Buchauer
+Bucciarelli
+Buccheri
+Bucaram
+Bubis
+Bubash
+Bubak
+Brzostek
+Brzezowski
+Bryton
+Brusuelas
+Brussell
+Bruschi
+Brundrett
+Brundin
+Brumet
+Bruley
+Bruk
+Brug
+Bruestle
+Brudner
+Bruccoleri
+Brozie
+Broxterman
+Brox
+Browy
+Brownle
+Browm
+Broward
+Brouwers
+Brousard
+Brought
+Brotherson
+Brotemarkle
+Brossoit
+Broscious
+Brooms
+Broomhall
+Brookshaw
+Brookhouse
+Bronchetti
+Broks
+Broida
+Brohl
+Broglie
+Brofft
+Broermann
+Broenneke
+Brodnex
+Brodka
+Brodish
+Brockelmeyer
+Brockberg
+Broch
+Broccoli
+Brobeck
+Broadstone
+Brittman
+Brislan
+Brisk
+Brisentine
+Bringhurst
+Brindel
+Brinda
+Brincks
+Brimeyer
+Brihm
+Brignolo
+Briglia
+Brighi
+Brient
+Bridenbaker
+Briddell
+Briante
+Brians
+Briagas
+Brevo
+Breu
+Bretto
+Bretthauer
+Breslauer
+Bresemann
+Brentari
+Brenning
+Brenhaug
+Brengettey
+Brenek
+Brendal
+Brenagh
+Breiling
+Breidenbaugh
+Brehant
+Bregel
+Bredeweg
+Bredehoft
+Breceda
+Braylock
+Brause
+Brauning
+Braulio
+Braukus
+Braucher
+Bratchett
+Brasseur
+Brasser
+Branstutter
+Branstad
+Branscombe
+Brannick
+Brandolini
+Brandly
+Brandenberg
+Brandeis
+Brandal
+Branciforte
+Brancheau
+Brancati
+Bramlette
+Bramlet
+Brakhage
+Braitman
+Braisted
+Bradfute
+Bracks
+Bracket
+Braccia
+Braam
+Bozzone
+Bozenski
+Bozard
+Boyson
+Boylston
+Boxwell
+Bowlen
+Bowdle
+Bowdich
+Boward
+Bovia
+Bovey
+Boven
+Bouza
+Bouwman
+Bouwkamp
+Boutiette
+Boursaw
+Bourret
+Bourgoyne
+Bounleut
+Bound
+Bouma
+Bouleris
+Bouler
+Boughman
+Boughamer
+Boudoin
+Boudewyns
+Botwinick
+Bottone
+Bottino
+Botticello
+Botten
+Bottaro
+Bottalico
+Bostel
+Boshes
+Boshard
+Bosell
+Boscarello
+Bory
+Borsari
+Borok
+Borodec
+Bornmann
+Bormuth
+Bormet
+Borling
+Borlace
+Borkin
+Borkenhagen
+Boreen
+Bordin
+Borcherding
+Boote
+Booras
+Boody
+Bonton
+Bontemps
+Bonomini
+Bonina
+Bonifer
+Bongartz
+Boness
+Bonefont
+Bonefield
+Bonder
+Bonde
+Bondanza
+Bonavia
+Bonamo
+Bonadurer
+Bomkamp
+Bolognia
+Bollich
+Bollacker
+Bolinsky
+Boldosser
+Boldon
+Bolda
+Bolado
+Boken
+Bok
+Boisselle
+Boisen
+Bois
+Bohs
+Bohnenblust
+Bohlig
+Bohinc
+Bogumil
+Bogie
+Boggioni
+Boggi
+Bogenschneide
+Bogema
+Boge
+Bogdanski
+Bogdanovich
+Boettner
+Boesiger
+Boesel
+Boensch
+Boele
+Boeken
+Boehning
+Boehlar
+Bodwell
+Bodreau
+Bodovsky
+Boda
+Boczar
+Boclair
+Bockemehl
+Bochenski
+Bochat
+Boch
+Boccio
+Bocchicchio
+Boccanfuso
+Bobzien
+Bobson
+Bobino
+Bobier
+Bobeck
+Bobak
+Boarts
+Boardwine
+Boaldin
+Boakye
+Boady
+Blunden
+Blumenstock
+Blovin
+Blouir
+Bloschichak
+Bloome
+Bloodough
+Blonder
+Blommer
+Blok
+Bloeser
+Blinks
+Blinka
+Bline
+Blickem
+Bleyl
+Blews
+Bless
+Blenner
+Bleimehl
+Blecker
+Bleasdale
+Bleakney
+Blatnick
+Blaski
+Blare
+Blanzy
+Blankumsee
+Blancett
+Blaich
+Blada
+Blackbum
+Bjorseth
+Bjorlin
+Bizzaro
+Bivin
+Bitetto
+Bisso
+Biskup
+Biskach
+Bisio
+Bisi
+Bishard
+Bisesi
+Bisaccia
+Birtcher
+Birrittella
+Birkhimer
+Birkey
+Biringer
+Biren
+Birdette
+Birak
+Bio
+Binker
+Bink
+Bingler
+Bingert
+Bingamon
+Bindas
+Bilson
+Billow
+Billon
+Billo
+Bille
+Bilis
+Bilich
+Biler
+Bilek
+Bilden
+Bilazzo
+Bila
+Bigus
+Biggart
+Biggar
+Bigaud
+Biesheuvel
+Biernacki
+Bierley
+Bierlein
+Bielefeldt
+Biedermann
+Biedenbender
+Biddulph
+Bicksler
+Bickes
+Bicek
+Bica
+Bibiano
+Biangone
+Bi
+Bezzo
+Bezdicek
+Beyt
+Beydler
+Bevelacqua
+Beuther
+Beucke
+Betzold
+Bettman
+Bettino
+Betterley
+Betancourth
+Bessel
+Beska
+Beschorner
+Berwald
+Berum
+Bertotti
+Bertorelli
+Bertoldo
+Bertolami
+Bertley
+Berteotti
+Bertaina
+Berstler
+Berniard
+Berndsen
+Bernadette
+Berlinski
+Berkstresser
+Berks
+Berkovich
+Berkoff
+Berkhimer
+Berkery
+Bergmark
+Berga
+Berfield
+Bereznak
+Beresky
+Berenger
+Berendzen
+Berendt
+Berczel
+Berch
+Berbes
+Berardinelli
+Beppu
+Benziger
+Benzie
+Benzango
+Benthall
+Bentancourt
+Bensberg
+Benno
+Bennin
+Bennes
+Benken
+Benike
+Benigni
+Benestad
+Bendtsen
+Bendis
+Bendig
+Bendetti
+Bendele
+Benasher
+Benack
+Bemben
+Belts
+Belrose
+Belnas
+Bellusci
+Belloso
+Bellizzi
+Bellinghausen
+Belliard
+Belletto
+Bellettiere
+Belko
+Belitz
+Belfanti
+Beldon
+Bekis
+Bejcek
+Beitler
+Beiser
+Beine
+Beiley
+Beierschmitt
+Behrle
+Behran
+Behlmer
+Behlke
+Beguelin
+Beghtol
+Beger
+Begeal
+Beezley
+Beesmer
+Beerer
+Beere
+Beerbohm
+Beenel
+Beelby
+Beecken
+Bedor
+Bede
+Beddows
+Beddow
+Beddia
+Becky
+Beckius
+Beckfield
+Beckem
+Becena
+Beavis
+Beaumonte
+Beauman
+Beauharnois
+Beaudine
+Beasly
+Beales
+Be
+Bazylewicz
+Bazner
+Bazel
+Baytos
+Bayton
+Bayt
+Baylock
+Bayird
+Baygents
+Baxa
+Bawner
+Bawden
+Bavelas
+Bauske
+Baumberger
+Baul
+Battuello
+Battig
+Batterman
+Battani
+Battaglino
+Batimon
+Bathke
+Baters
+Batch
+Batas
+Batara
+Batala
+Bastine
+Bassani
+Bassali
+Baskind
+Baseman
+Basehore
+Basara
+Barze
+Barwell
+Barut
+Baruffa
+Bartlome
+Bartin
+Barthol
+Barthell
+Barters
+Barswell
+Barshaw
+Barrigan
+Barria
+Barrasa
+Barraco
+Barnthouse
+Barnt
+Barmes
+Barkhimer
+Barios
+Bario
+Barino
+Barie
+Barick
+Barfuss
+Barfknecht
+Barer
+Bareford
+Bardis
+Barcley
+Barchick
+Barcena
+Barbur
+Barbor
+Barbin
+Barben
+Barbella
+Barbaglia
+Baransky
+Baragan
+Baquiran
+Banzhaf
+Banter
+Bankowski
+Banet
+Bandt
+Banaszek
+Banana
+Balque
+Balowski
+Ballog
+Ballina
+Ballensky
+Ballato
+Baliga
+Baldomero
+Balden
+Balde
+Baldassare
+Balbontin
+Balbas
+Balassi
+Balandran
+Bakkala
+Bakhshian
+Bakerville
+Bakaler
+Bajaj
+Baites
+Baisten
+Bairam
+Bailard
+Baierl
+Baichan
+Bai
+Bahrs
+Bagozzi
+Bagni
+Bagnato
+Baglione
+Baggio
+Baggesen
+Baggenstoss
+Bagan
+Baessler
+Baerman
+Baerlocher
+Badgero
+Baddour
+Badami
+Baculpo
+Bacio
+Bacigalupo
+Bachta
+Bachar
+Bacchi
+Babrow
+Babonis
+Babish
+Babicke
+Babeu
+Baab
+Azzopardi
+Azore
+Azen
+Aykroid
+Axon
+Axelrad
+Awkard
+Awender
+Avon
+Avirett
+Averitte
+Averbeck
+Avellano
+Avary
+Auwaerter
+Autrano
+Auteri
+Austgen
+Ausdemore
+Aurich
+Aumen
+Auler
+Augustyniak
+Augliano
+Aughtman
+Aue
+Auduong
+Aucter
+Attianese
+Atiles
+Athas
+Asturias
+Astrup
+Astley
+Assante
+Aspden
+Aspacio
+Asley
+Asleson
+Askvig
+Askegren
+Askam
+Ashmen
+Ashauer
+Asfour
+Aschoff
+Aschim
+Aschan
+Asal
+Arzo
+Arvesen
+Arrow
+Arrocha
+Arris
+Arribas
+Arquitt
+Arone
+Aroche
+Arnt
+Arnoux
+Arnoldi
+Arning
+Arnholt
+Arndorfer
+Armson
+Arment
+Arlotta
+Arlinghaus
+Arlia
+Arkema
+Arizaga
+Arisumi
+Aristide
+Aris
+Arif
+Ariano
+Arguilez
+Argudo
+Argrow
+Argiro
+Argetsinger
+Arfman
+Arenburg
+Aredondo
+Area
+Ardry
+Ardner
+Ardizone
+Arcudi
+Arcizo
+Arcila
+Archilla
+Archangel
+Arcega
+Arbucci
+Arato
+Arano
+Aran
+Aragan
+Apostol
+Apolito
+Apland
+Apkin
+Aperges
+Apalategui
+Apaez
+Anzora
+Antonsen
+Antolos
+Antolini
+Antman
+Anter
+Anspaugh
+Anselm
+Annonio
+Annichiarico
+Annibale
+Annarumo
+Anliker
+Ankrapp
+Ankenman
+Anhorn
+Angton
+Angrisano
+Angon
+Angolo
+Angleton
+Anglebrandt
+Anglea
+Anglade
+Angilletta
+Angeron
+Angelotti
+Angelbeck
+Angela
+Anez
+Andueza
+Andrulis
+Andronis
+Andreu
+Andreoni
+Andert
+Anderlik
+Anauo
+Anastasiades
+Ananias
+Anand
+Amuso
+Amrich
+Amr
+Amour
+Amoss
+Amorosi
+Amoako
+Amoah
+Ammirato
+Ammar
+Amirian
+Amiot
+Amidi
+Ameduri
+Amderson
+Ambuehl
+Amass
+Amanza
+Amadio
+Alwang
+Alwan
+Alvine
+Alvarran
+Alvarracin
+Alvanez
+Aluqdah
+Altshuler
+Altonen
+Altmiller
+Altken
+Altiery
+Althiser
+Altaras
+Alstrom
+Alstad
+Alsbury
+Alsberry
+Alquijay
+Alpha
+Alonza
+Aloia
+Alnas
+Almerico
+Almenar
+Almen
+Allwood
+Allstott
+Allridge
+Alleva
+Allenson
+Allenbaugh
+Allegretta
+Allegra
+Allbritten
+Allara
+Allamon
+Alken
+Alizadeh
+Alirez
+Alires
+Aline
+Alim
+Algire
+Algier
+Algien
+Alfonsi
+Alexy
+Alexnder
+Alessandroni
+Alert
+Alemany
+Aleksey
+Alderton
+Alderfer
+Aldava
+Aldapa
+Alconcel
+Albornoz
+Albini
+Albergotti
+Alben
+Albea
+Albang
+Alario
+Alamilla
+Alalem
+Akoni
+Akles
+Akande
+Akamine
+Ajasin
+Aiyer
+Aihara
+Ahrendes
+Aherns
+Aharoni
+Agunos
+Aguliar
+Aguillar
+Agudo
+Agoras
+Agnor
+Agni
+Agers
+Agel
+Aery
+Aerts
+Adon
+Adessa
+Aderson
+Aderman
+Adema
+Adelsberg
+Adelblue
+Adel
+Addiego
+Adas
+Adamcik
+Acquilla
+Ackmann
+Achterhof
+Achane
+Abuhl
+Abrial
+Abreau
+Aboulahoud
+Aboudi
+Ablao
+Abilez
+Abete
+Aberson
+Abelman
+Abelardo
+Abedelah
+Abdulmateen
+Abato
+Aas
+Aarestad
+Aanenson
+Zymowski
+Zyla
+Zybia
+Zwolski
+Zwigart
+Zuwkowski
+Zurovec
+Zurkuhlen
+Zuppa
+Zunich
+Zumpfe
+Zumalt
+Zulkowski
+Zulfer
+Zugg
+Zuerlein
+Zuehls
+Zuckerberg
+Zuchelkowski
+Zucchetto
+Zucca
+Zubrowski
+Zubizarreta
+Zsadanyi
+Zrake
+Zotti
+Zosel
+Zoltek
+Zolla
+Zogopoulos
+Zogby
+Zmek
+Zitzmann
+Zitzelberger
+Zirker
+Zinzow
+Zimick
+Zimerman
+Zilk
+Zigomalas
+Ziesman
+Ziernicki
+Zierke
+Zierk
+Zierenberg
+Zierden
+Ziems
+Zieger
+Ziebert
+Zicafoose
+Zic
+Zibell
+Ziada
+Ziad
+Zhen
+Zetzer
+Zetino
+Zerphey
+Zercher
+Zeran
+Zephyr
+Zelonis
+Zellinger
+Zelko
+Zeliff
+Zeleznik
+Zekria
+Zeidman
+Zehrer
+Zehrbach
+Zeherquist
+Zehender
+Zegar
+Zega
+Zechiel
+Zeccardi
+Zebracki
+Zeavala
+Zbierski
+Zaza
+Zayicek
+Zawistowski
+Zawasky
+Zavitz
+Zaverl
+Zavcedo
+Zavattieri
+Zavacky
+Zausch
+Zatorski
+Zarrabi
+Zarlingo
+Zarin
+Zarillo
+Zaren
+Zapel
+Zapatero
+Zantow
+Zant
+Zannini
+Zangger
+Zanfardino
+Zanardi
+Zan
+Zampella
+Zamoro
+Zamborano
+Zambelli
+Zalamea
+Zajdel
+Zais
+Zahourek
+Zaharek
+Zagulski
+Zagacki
+Zadina
+Zaczek
+Zachter
+Zachariah
+Zacchini
+Zabenko
+Zabbo
+Yuska
+Yuscak
+Yurovic
+Yurek
+Yunes
+Yumas
+Yuk
+Yudell
+Ysaguirre
+Yray
+Yozzo
+Yovan
+Youssefi
+Yousko
+Younghans
+Youmon
+Youla
+Yotter
+Yoshi
+Yoseph
+Yorck
+Yono
+Yoneoka
+Yonashiro
+Yomes
+Yokel
+Yoest
+Ynocencio
+Yewell
+Yetzer
+Yetsko
+Yerty
+Yeropoli
+Yerka
+Yergin
+Yenor
+Yem
+Yeley
+Yearego
+Yeakel
+Yazzle
+Yazzi
+Yazdani
+Yaws
+Yasika
+Yarwood
+Yarris
+Yaroch
+Yarmitsky
+Yara
+Yantzi
+Yannucci
+Yannayon
+Yannantuono
+Yankovski
+Yankovitch
+Yandow
+Yanchik
+Yanagihara
+Yanagida
+Yanacek
+Yamanoha
+Yamaki
+Yalon
+Yaklin
+Yake
+Yaiva
+Yaish
+Yahne
+Yafuso
+Yafaie
+Yacullo
+Yacovone
+Yacoub
+Xyong
+Xayasith
+Wyze
+Wyrostek
+Wynes
+Wyker
+Wygal
+Wybenga
+Wurz
+Wung
+Wueste
+Wubnig
+Wubbena
+Wubben
+Wrzesien
+Wrynn
+Wrightington
+Wride
+Wreyford
+Woytowich
+Woytek
+Wosick
+Workowski
+Worell
+Wordlow
+Worchester
+Wooward
+Woolhiser
+Woodlin
+Woodka
+Woodbeck
+Woodal
+Wondoloski
+Wonderling
+Wolsdorf
+Wolper
+Wollert
+Wollenburg
+Woline
+Wolfing
+Wolfensperger
+Wolbrecht
+Wojnowski
+Wojewoda
+Wojdak
+Wohlfeil
+Wohlert
+Woge
+Woelfl
+Wodicka
+Wobser
+Wobbe
+Wnukowski
+Wnorowski
+Wmith
+Wlodarek
+Wiza
+Witucki
+Wittrup
+Wittnebel
+Witthoeft
+Wittenbrink
+Wittbrodt
+Witkowsky
+Wisnowski
+Wisely
+Wirtzfeld
+Wirfs
+Wipfli
+Winterberg
+Winslette
+Winscott
+Winnicki
+Winnen
+Winik
+Wingeier
+Windsheimer
+Windrow
+Windhorst
+Windfield
+Windauer
+Wincapaw
+Win
+Wimbrow
+Wimble
+Wilund
+Wilshusen
+Wilsen
+Willock
+Willmert
+Willies
+Williemae
+Williamis
+Willia
+Willi
+Willeto
+Willborn
+Wilkus
+Wilkson
+Wilkoff
+Wildridge
+Wilczak
+Wilcut
+Wiklund
+Wiggan
+Wigand
+Wig
+Wiesemann
+Wieseman
+Wiersteiner
+Wienberg
+Wielock
+Wielgasz
+Wiegard
+Wiedrich
+Wiederholt
+Wieben
+Widjaja
+Widera
+Wide
+Wicklin
+Wickersheim
+Wiborg
+Wiatrowski
+Why
+Whittum
+Whittinghill
+Whittenbeck
+Whitiker
+Whitey
+Whiter
+Whitelightnin
+Whitcome
+Whisted
+Whirlow
+Whiles
+Whilden
+Whetzell
+Whelihan
+Wheeldon
+Wheater
+Whaltey
+Weynand
+Weyker
+Weydert
+Weuve
+Wetzstein
+Wetzell
+Westler
+Westermeier
+Westermark
+Westermann
+Westerhoff
+Westbrooke
+Weske
+Weser
+Werst
+Werremeyer
+Wernsman
+Wernex
+Wern
+Werme
+Werline
+Werk
+Wergin
+Werdlow
+Werderman
+Went
+Wensman
+Wenske
+Wendorff
+Welzel
+Weltha
+Wellinghoff
+Welding
+Weit
+Weissenbach
+Weispfenning
+Weismantle
+Weisbecker
+Weirauch
+Weinzierl
+Weinrib
+Weinland
+Weinfurter
+Weinburg
+Weiher
+Weig
+Weidower
+Weicht
+Weibe
+Wehking
+Weglage
+Wegiel
+Wedige
+Weckwerth
+Weatherington
+Weasel
+Weant
+Wealer
+Weagraff
+Weader
+Wayts
+Wayson
+Waymon
+Waygood
+Wayford
+Waychowsky
+Waverly
+Wattigny
+Watsky
+Watry
+Wates
+Watah
+Wasurick
+Wassam
+Waskom
+Waskin
+Washum
+Washpun
+Washler
+Waser
+Warzybok
+Warstler
+Warrilow
+Warran
+Waroway
+Warntz
+Warnberg
+Warmka
+Warmbrod
+Warlow
+Warlock
+Warde
+War
+Wapp
+Wantuck
+Wannlund
+Wannarka
+Wanko
+Wandell
+Walund
+Waltos
+Waltho
+Walstrum
+Walrod
+Walper
+Waln
+Wallwork
+Wallo
+Wallman
+Walliser
+Wallie
+Wallenbrock
+Wallau
+Walka
+Walizer
+Walgren
+Waley
+Walen
+Waldroop
+Walderon
+Wal
+Wakeford
+Waitz
+Waiss
+Waisanen
+Wais
+Wainkrantz
+Wahn
+Wahdan
+Wahba
+Wagnor
+Waggy
+Wagemann
+Wagatsuma
+Waffenschmidt
+Waegner
+Waddups
+Waddles
+Wadas
+Wacht
+Waas
+Waaga
+Vuoso
+Vukelj
+Vriens
+Vredeveld
+Vrbas
+Vranicar
+Vovak
+Votsmier
+Vostal
+Vorsburgh
+Vornes
+Vopava
+Vonseeger
+Vonschriltz
+Vonholt
+Vongsamphanh
+Vongkhamphanh
+Vongkhamchanh
+Vonfelden
+Voner
+Vondrasek
+Vondracek
+Vonderhaar
+Vonderahe
+Vonbank
+Volpone
+Volmar
+Vollmers
+Vollette
+Volinsky
+Volek
+Volbert
+Vojna
+Voigtlander
+Vogelzang
+Voeltz
+Voelkerding
+Vocelka
+Vljeric
+Vleming
+Vlchek
+Vizzi
+Vixayack
+Vixay
+Vivyan
+Vivion
+Vitrano
+Vitez
+Vitellaro
+Visounnaraj
+Visick
+Viscosi
+Virostko
+Virgile
+Virgadamo
+Virant
+Vintila
+Vinti
+Vint
+Vilven
+Vilt
+Villnave
+Villescaz
+Ville
+Villasis
+Villaplana
+Villao
+Villanveua
+Villanvera
+Villandry
+Villamayor
+Villamarin
+Villaluz
+Villaluazo
+Villaire
+Villacrusis
+Vilegas
+Vildosola
+Viker
+Vijil
+Vijayan
+Vigneau
+Vigilo
+Vigiano
+Vieu
+Vietzke
+Vierk
+Viengxay
+Vieau
+Vidas
+Vidaca
+Vicuna
+Vicueroa
+Vicenteno
+Vias
+Viard
+Viano
+Viale
+Viafara
+Vezza
+Vevea
+Vetterkind
+Vetterick
+Veto
+Vessar
+Vesperas
+Vesley
+Verwers
+Verunza
+Verso
+Versage
+Verrue
+Verrone
+Verrastro
+Verplanck
+Verone
+Vernazza
+Verlinden
+Verlin
+Verkuilen
+Verfaillie
+Venzor
+Venturelli
+Venskoske
+Venning
+Venneman
+Veneri
+Vendig
+Vence
+Veltkamp
+Velthuis
+Velovic
+Veller
+Velky
+Velega
+Velardes
+Veksler
+Veitinger
+Vehrenkamp
+Vegerano
+Vedovelli
+Veasman
+Vbiles
+Vautier
+Vaulet
+Vatterott
+Vasudevan
+Vasos
+Vasek
+Vasallo
+Varquez
+Varquera
+Varoz
+Varone
+Varisco
+Varieur
+Varanda
+Vanzie
+Vanwyck
+Vanwhy
+Vanweerd
+Vanwechel
+Vanvuren
+Vanvorst
+Vanveldhuize
+Vanuden
+Vantuyle
+Vantull
+Vansteenhuyse
+Vansteenberg
+Vanson
+Vansise
+Vanschoor
+Vanschoiack
+Vanrossum
+Vanosdol
+Vanos
+Vanorsouw
+Vanoni
+Vannuck
+Vanlinden
+Vanlier
+Vanlaere
+Vaninetti
+Vanhove
+Vanhoutte
+Vanhoecke
+Vanheusen
+Vanhamme
+Vanham
+Vangordon
+Vaneekelen
+Vandonsel
+Vandevanter
+Vandesande
+Vandernoot
+Vanderjagt
+Vanderiet
+Vanderhurst
+Vanderbie
+Vandawalker
+Vandaele
+Vanblaricum
+Vanbeveren
+Vanamerongen
+Vanamburgh
+Vanalstin
+Valtas
+Valme
+Vallow
+Vallotton
+Valliant
+Vallegos
+Vallar
+Valladores
+Valerino
+Valeriani
+Valela
+Valdo
+Valant
+Valado
+Vajnar
+Vais
+Vagnier
+Vadlamudi
+Vactor
+Vaccarello
+Vacarro
+Uzzo
+Uutela
+Utzig
+Useted
+Urtz
+Urtiz
+Urtiaga
+Urteaga
+Urquides
+Urmston
+Urmos
+Urbany
+Urbaez
+Uptmor
+Upole
+Uphold
+Uoy
+Unverzagt
+Unvarsky
+Unterseher
+Unterman
+Unglesbee
+Underdue
+Uncapher
+Umeh
+Ulven
+Ulvan
+Ulshafer
+Ulsamer
+Uljevic
+Ulbricht
+Ulabarro
+Ujano
+Uimari
+Uihlein
+Ugolini
+Uglum
+Ufford
+Ueckert
+Udani
+Uchiyama
+Ubl
+Ubaldo
+Tyrie
+Tyndal
+Tyms
+Tylwalk
+Tyeryar
+Twilligear
+Twidwell
+Twardy
+Tuzzio
+Tutterow
+Tutaj
+Turziano
+Turzak
+Turtura
+Turtle
+Turrietta
+Turns
+Turnell
+Turneer
+Turnbill
+Turello
+Turbacuski
+Tupaj
+Tupacyupanqui
+Tuomi
+Tuomala
+Tuohey
+Tuning
+Tumolo
+Tuman
+Tullar
+Tulino
+Tuggerson
+Tuckerson
+Tucke
+Tuchy
+Tucek
+Tucciarone
+Tuamoheloa
+Tuai
+Tua
+Tsu
+Tsironis
+Tsing
+Tsiatsos
+Tsemetzis
+Tscrious
+Tsau
+Tsasie
+Tsakonas
+Trypaluk
+Trygg
+Truxell
+Truver
+Trusso
+Trush
+Trusello
+Truocchio
+Truncellito
+Trumps
+Trumper
+Trumbley
+Trulli
+Truhe
+Truglia
+Trufin
+Trudnowski
+Trudics
+Trudgeon
+Trucks
+Trucker
+Troyano
+Troyani
+Trouser
+Trotty
+Tronaas
+Tromley
+Tromburg
+Troller
+Trojecki
+Trojahn
+Troike
+Troidl
+Troge
+Trofholz
+Trochesset
+Trish
+Trio
+Trinkley
+Trinkl
+Tringham
+Trindle
+Trimnell
+Trilli
+Trill
+Triguro
+Trigueros
+Triece
+Trider
+Trexel
+Trewin
+Trewhitt
+Treuter
+Treutel
+Trettin
+Trett
+Treso
+Trenton
+Trentini
+Trenholme
+Tremel
+Trell
+Tregan
+Trecarichi
+Trbovich
+Traverse
+Traunfeld
+Trapanese
+Tramp
+Tramm
+Trajillo
+Trahin
+Traher
+Tradup
+Toyne
+Toyama
+Townzen
+Towber
+Toussiant
+Tousom
+Tourtelotte
+Touma
+Toulmin
+Touhy
+Tottingham
+Totter
+Tott
+Totosz
+Toti
+Tota
+Tostanoski
+Toso
+Tory
+Torreson
+Torreon
+Torrell
+Torralva
+Torno
+Torngren
+Tornese
+Tordsen
+Torbit
+Torbeck
+Toppins
+Toppen
+Toppah
+Topolinski
+Toplk
+Topliss
+Toplin
+Topinka
+Topi
+Toomsen
+Tools
+Toof
+Too
+Tonic
+Toniatti
+Toni
+Tongren
+Tonche
+Tonas
+Tomsick
+Tomsche
+Tomopoulos
+Tomkowicz
+Tomasko
+Toliongco
+Toleston
+Tokunaga
+Tokita
+Tohonnie
+Tognetti
+Toevs
+Todora
+Todahl
+Tod
+Tocher
+Tocchio
+Tobosa
+Tobiason
+Tjepkema
+Tizon
+Tixier
+Tiwald
+Tittl
+Tisue
+Tisinger
+Tisa
+Tirona
+Tiro
+Tirk
+Tirino
+Tiotuico
+Tinnea
+Tinin
+Timone
+Timber
+Tilleman
+Tille
+Tiley
+Tijing
+Tigg
+Tiffner
+Tietjens
+Tieger
+Tidrington
+Tidrick
+Tibwell
+Tibolla
+Tibbit
+Tiangco
+Tian
+Thyfault
+Thurstonson
+Thundercloud
+Thuman
+Thrun
+Thrill
+Thorsten
+Thornquist
+Thorner
+Thormina
+Thormer
+Thoran
+Thomspon
+Thoeny
+Thoennes
+Thoele
+Thoby
+Thillet
+Thiesse
+Thibedeau
+Theuner
+Thessing
+Therurer
+Thero
+Theo
+Themot
+Them
+Thein
+Theim
+Theiling
+Theesfeld
+Theaker
+Thaniel
+Thamphia
+Thammorongsa
+Thalheimer
+Thain
+Thaemert
+Thackxton
+Thackrey
+Thackery
+Teyler
+Tewmey
+Tevada
+Tetz
+Tetteh
+Tetro
+Tetreau
+Testman
+Tessner
+Tesoriero
+Tesnow
+Tesauro
+Tersteeg
+Terrett
+Terrero
+Terrence
+Terrall
+Terr
+Terkelsen
+Terbush
+Teranishi
+Tepperberg
+Tentler
+Tenor
+Tenharmsel
+Tengwall
+Tenerowicz
+Tenebruso
+Tendick
+Tencer
+Ten
+Temoshenka
+Telman
+Tellinghuisen
+Telega
+Telchik
+Tejeiro
+Teitel
+Teichrow
+Teichmiller
+Tegtmeier
+Tegenkamp
+Teet
+Teeples
+Teepe
+Tebow
+Tebbetts
+Tebbe
+Tease
+Teach
+Tayo
+Taymon
+Taylan
+Taydus
+Tavolario
+Taves
+Tauteoli
+Tatu
+Tatsak
+Tatnall
+Tates
+Tasto
+Tasse
+Tashman
+Tartar
+Tarsis
+Tarris
+Tarricone
+Tarran
+Tarner
+Tarbor
+Tarbet
+Tarasuik
+Taraschke
+Taps
+Tappis
+Tapio
+Tapat
+Tapales
+Tapaha
+Taomoto
+Tanzosch
+Tanzman
+Tanweer
+Tanoue
+Tanori
+Tanon
+Tannazzo
+Tanker
+Tanke
+Tango
+Tanen
+Tandon
+Tandetzke
+Tancer
+Tamminen
+Tamiya
+Tameron
+Talladino
+Taliulu
+Talburt
+Talboti
+Talat
+Talamas
+Takiguchi
+Takenaka
+Tak
+Tahir
+Tagliente
+Taglialatela
+Tagge
+Tagami
+Tafuri
+Tafreshi
+Tacderen
+Taccariello
+Tacata
+Tacadina
+Tablada
+Tabet
+Taberski
+Tabbaa
+Taake
+Szypowski
+Szynkowicz
+Szymula
+Szychowski
+Szwarc
+Szuszkiewicz
+Szumny
+Szumilas
+Szumiesz
+Szuch
+Szuba
+Sznejkowski
+Szmidt
+Szlosek
+Szigethy
+Szenasi
+Szczurek
+Szczesniak
+Szalankiewicz
+Szalai
+Szal
+Szaflarski
+Syrstad
+Syrop
+Synowiec
+Synakowski
+Symore
+Symon
+Syddall
+Sybounheuan
+Swonke
+Swisshelm
+Swiller
+Swenton
+Swell
+Sweley
+Sweger
+Swefford
+Sweere
+Swee
+Swedeen
+Sweazey
+Swearngen
+Swaynos
+Swatloski
+Swatek
+Swary
+Swartley
+Swarr
+Swarn
+Swarb
+Swarat
+Swanzy
+Swantner
+Swantko
+Swanteck
+Swanick
+Swaine
+Swadling
+Svob
+Svensen
+Sutt
+Suto
+Sutherburg
+Susmilch
+Susla
+Susko
+Susan
+Surridge
+Surran
+Surkamer
+Suon
+Suominen
+Suneson
+Sundman
+Sumstad
+Sumruld
+Sumey
+Sumbera
+Sumaran
+Sultaire
+Sully
+Sulloway
+Sulkowski
+Sulc
+Sukut
+Sukup
+Sukovich
+Suihkonen
+Suga
+Suffern
+Sueyoshi
+Suet
+Suennen
+Suellentrop
+Sueda
+Suddath
+Succop
+Sub
+Sualevai
+Styler
+Stvictor
+Stuzman
+Stusse
+Sturwold
+Sturino
+Sturiale
+Sturdnant
+Stupke
+Stumm
+Stumb
+Stukel
+Stufflebean
+Stuever
+Stuessy
+Stuedemann
+Stueckrath
+Stueck
+Studwell
+Stubler
+Stubbert
+Strzyzewski
+Strzelczyk
+Strutynski
+Struckmann
+Struber
+Strow
+Stropus
+Strople
+Stroot
+Strohecker
+String
+Strimel
+Stright
+Striffler
+Stridiron
+Stricklan
+Strem
+Streller
+Strekas
+Strek
+Streitz
+Streitenberge
+Strech
+Streat
+Strazzullo
+Strawberry
+Stratter
+Strathmann
+Strassell
+Strassberg
+Strangstalien
+Stoyanov
+Stouten
+Stoutamyer
+Stotelmyer
+Stoskopf
+Storton
+Storbeck
+Stoppenbach
+Stoot
+Stoor
+Stonewall
+Stonefield
+Stolzenberg
+Stollsteimer
+Stokel
+Stohs
+Stohrer
+Stofferahn
+Stoermer
+Stoen
+Stoecklin
+Stockhoff
+Stockburger
+Stoakley
+Stoa
+Stlucien
+Stitz
+Stittgen
+Stitch
+Stires
+Stippich
+Stinser
+Stinemetz
+Stinde
+Stinar
+Stimus
+Stiliner
+Stilgenbauer
+Stifflemire
+Stickfort
+Sticher
+Stibb
+Stewardson
+Stevison
+Steube
+Sternod
+Sterger
+Steptore
+Steppig
+Stepleton
+Stephanski
+Stephano
+Stepchinski
+Stepanik
+Stepaniak
+Stenslien
+Stenslie
+Stengle
+Stengele
+Stendal
+Stempert
+Steman
+Stelmach
+Steitzer
+Steinworth
+Steinway
+Steins
+Steinour
+Steinmiller
+Steinhouse
+Steinhour
+Steinger
+Steindorf
+Steinau
+Steinacker
+Stegmann
+Steff
+Stefansky
+Steensland
+Steenrod
+Steenland
+Steeby
+Stech
+Stealy
+Steagell
+Steadings
+Steach
+Stawasz
+Stavsvick
+Stavrides
+Stavish
+Stathes
+State
+Stassinos
+Stasser
+Stasio
+Stasa
+Starzynski
+Starritt
+Starring
+Starnold
+Starchman
+Starch
+Starace
+Stapelton
+Stanuszek
+Stanovich
+Stankovic
+Stankey
+Stanislaw
+Staniforth
+Stanier
+Stangarone
+Stanganelli
+Standlee
+Standerwick
+Standback
+Stancombe
+Stancer
+Stancato
+Stammel
+Stambough
+Stallones
+Stakelin
+Stagnitto
+Stafiej
+Staffon
+Staffieri
+Staffen
+Stade
+Stachniw
+Stachnik
+Stacer
+Staber
+Stabell
+Staback
+Staadt
+Spunt
+Spueler
+Spruit
+Spruel
+Spriggins
+Spratlen
+Sprain
+Sprafka
+Sportsman
+Sports
+Sporle
+Spoerl
+Spoerer
+Splonskowski
+Splinter
+Splane
+Spizzirri
+Spinoso
+Spinka
+Spiney
+Spine
+Spindola
+Spindle
+Spinas
+Spilski
+Spielmaker
+Spiegle
+Spevacek
+Sperrey
+Sperger
+Sperduti
+Speranza
+Sperandeo
+Spender
+Spena
+Spella
+Speith
+Speis
+Speiden
+Speidell
+Speese
+Specter
+Speake
+Speagle
+Spaun
+Spara
+Spanton
+Spanswick
+Spannbauer
+Spana
+Spaide
+Spadlin
+Sowash
+Sovey
+Sovak
+Souvannavong
+Souvannarith
+Souvannakhiry
+Souser
+Soulek
+Soukkhavong
+Soucek
+Sottosanti
+Sotlar
+Sotak
+Sossong
+Sosso
+Sosinsky
+Soscia
+Sorotzkin
+Sorokin
+Sorman
+Sorgatz
+Soren
+Soravilla
+Sor
+Soprych
+Sopata
+Soorus
+Sookoo
+Sonnenburg
+Sonkens
+Sondrini
+Sondelski
+Somsana
+Sommerdorf
+Sommella
+Solverson
+Soltren
+Soltes
+Solonika
+Solomons
+Sollock
+Sollman
+Solle
+Solimeno
+Soliece
+Solgovic
+Soldow
+Solas
+Solarz
+Sokorai
+Sokolik
+Soisson
+Sohrabi
+Soho
+Sogol
+Soga
+Sofka
+Sodomka
+Sodachanh
+Sochocki
+Socci
+Sobrowski
+Sobrino
+Soboleski
+Soberano
+Sobba
+Sobania
+Soans
+Snuffer
+Snowdon
+Snowdeal
+Snoderly
+Snock
+Snitker
+Snith
+Sniff
+Snedeger
+Snearly
+Snachez
+Smurthwaite
+Smolski
+Smithmyer
+Smithen
+Smithberger
+Smisek
+Smily
+Smiglewski
+Smietana
+Smialowski
+Smeltz
+Smelko
+Smeenk
+Smedsrud
+Smayda
+Smaw
+Smarsh
+Smalt
+Smalarz
+Slutzky
+Sluis
+Sloup
+Slotkin
+Slosek
+Sloon
+Slomski
+Slocombe
+Slockbower
+Slisz
+Slinsky
+Slicer
+Sleek
+Slayman
+Slavis
+Slatin
+Slanina
+Slagel
+Sladky
+Sladek
+Skyberg
+Skwara
+Skursky
+Skurski
+Skura
+Skrobacki
+Skretowicz
+Skorepa
+Skomo
+Sknerski
+Skinsacos
+Skillom
+Skillen
+Skibosh
+Skibisky
+Skewis
+Skene
+Skender
+Skalecki
+Skafec
+Sixon
+Sivia
+Sivert
+Sitto
+Sita
+Sissman
+Sisneroz
+Siskey
+Sischo
+Sirwet
+Sirucek
+Sirrine
+Sirnio
+Siriani
+Sirek
+Sippial
+Sionesini
+Sioma
+Sinkiewicz
+Sininger
+Singuefield
+Sings
+Singhisen
+Singeltary
+Singco
+Siner
+Sindt
+Sindorf
+Sindoni
+Sindel
+Simzer
+Simunek
+Simplot
+Simpelo
+Simonetta
+Simonett
+Simoneavd
+Simmelink
+Simlick
+Simkowitz
+Simino
+Simers
+Simer
+Simcic
+Simank
+Silverwood
+Silverhorn
+Silquero
+Sillitti
+Sillery
+Silla
+Silker
+Silerio
+Silagy
+Silago
+Sikorra
+Sikkila
+Sikel
+Sikat
+Sikander
+Sigworth
+Signorino
+Sigafoos
+Siewers
+Sievel
+Sierzenga
+Sierer
+Siepker
+Siena
+Sien
+Siegfreid
+Siegers
+Siefkes
+Siefferman
+Siebel
+Sidles
+Side
+Siddiq
+Sida
+Sickmeir
+Sickendick
+Sichler
+Sicheneder
+Sichel
+Siangco
+Siad
+Shymske
+Shutte
+Shutes
+Shurkus
+Shumay
+Shukert
+Shuhi
+Shuga
+Shuckhart
+Shryer
+Shroeder
+Shrimplin
+Shrier
+Shrefler
+Shrake
+Shoyer
+Showden
+Shouts
+Shoto
+Shonts
+Shoeman
+Shoddie
+Shirilla
+Shird
+Shirai
+Shipwash
+Shiplet
+Shipler
+Shintani
+Shinney
+Shinko
+Shindorf
+Shimonishi
+Shimanuki
+Shiller
+Shiiba
+Shigemitsu
+Shigematsu
+Shifley
+Shifflette
+Shiever
+Shido
+Shidemantle
+Shidel
+Shibahara
+Shey
+Shevenell
+Shetz
+Sheskey
+Sherratt
+Sherif
+Sherfy
+Sherbo
+Shepp
+Shenberger
+Shenassa
+Shemper
+Sheltrown
+Shellum
+Shellnut
+Shellhorn
+Shellgren
+Shelenberger
+Sheive
+Sheasby
+Shearier
+Shearhart
+Shawler
+Shawaiki
+Shaull
+Shau
+Shatt
+Sharratt
+Sharrai
+Sharpsteen
+Sharpey
+Sharley
+Shariff
+Shariat
+Sharar
+Shapin
+Shansky
+Shannonhouse
+Shangraw
+Shammaa
+Shamapande
+Shalam
+Shaker
+Shahinian
+Shaginaw
+Shaggy
+Shafto
+Shafi
+Shaer
+Shae
+Shadix
+Shadburn
+Sfera
+Sfatcu
+Seymoure
+Sey
+Sewester
+Severyn
+Seutter
+Seuss
+Seufer
+Settecase
+Sespinosa
+Servey
+Servano
+Serum
+Sertuche
+Sert
+Serro
+Serret
+Serre
+Sermon
+Sermania
+Sergovia
+Seremet
+Serabia
+Ser
+Sephton
+Sep
+Senta
+Sensenbach
+Senneker
+Senk
+Senion
+Senemounnarat
+Seneker
+Semo
+Semenick
+Seltrecht
+Sellar
+Seliski
+Selis
+Seligmann
+Selia
+Selestewa
+Selem
+Sele
+Selca
+Selbert
+Selbe
+Sekerak
+Sejkora
+Seiz
+Seiver
+Seirer
+Seilhymer
+Seiley
+Seiger
+Seigart
+Seifts
+Seiffert
+Seidle
+Seide
+Seiberlich
+Segota
+Segobia
+Seewald
+Seepersaud
+Seen
+Sedy
+Sedtal
+Sedotal
+Sedler
+Sedlachek
+Secreto
+Secora
+Secky
+Seckington
+Sebestyen
+Sebers
+Searchwell
+Searchfield
+Searcey
+Seanor
+Sean
+Seamen
+Sealander
+Seaford
+Scullion
+Scrudato
+Scronce
+Scrobola
+Scribellito
+Scozzari
+Scoresby
+Scolnik
+Scoh
+Scoble
+Sclavi
+Sciuto
+Scisco
+Scigliano
+Scieszka
+Scierka
+Scibetta
+Sciavillo
+Sciarini
+Sciancalepore
+Schwuchow
+Schwoyer
+Schwoerer
+Schwien
+Schwetz
+Schwertfager
+Schwentker
+Schwent
+Schwendinger
+Schwemm
+Schweiner
+Schwarzenberg
+Schwartzer
+Schwarten
+Schwanebeck
+Schwanbeck
+Schwallie
+Schwald
+Schuyleman
+Schustrich
+Schurer
+Schuppenhauer
+Schumucker
+Schumans
+Schuiling
+Schueth
+Schuckert
+Schuchmann
+Schuble
+Schub
+Schroy
+Schromen
+Schroeppel
+Schroedel
+Schreur
+Schreimann
+Schrecker
+Schouweiler
+Schou
+Schornick
+Schoreplum
+Schooling
+School
+Schoo
+Schontz
+Schoninger
+Schoneck
+Schone
+Schonaerts
+Schomberg
+Schollmeier
+Schoepflin
+Schoenegge
+Schoeneck
+Schoeller
+Schoebel
+Schnitman
+Schnetter
+Schnelzer
+Schneidmiller
+Schnair
+Schnabl
+Schmuff
+Schmoldt
+Schmider
+Schmeer
+Schlussel
+Schlissel
+Schlett
+Schlesner
+Schlesener
+Schlepphorst
+Schlepp
+Schlechten
+Schlaack
+Schiveley
+Schirm
+Schimanski
+Schilmoeller
+Schille
+Schilawski
+Schiffner
+Schiffert
+Schiedler
+Schickler
+Schiappa
+Scheuring
+Scheule
+Schepker
+Schenz
+Schenkelberg
+Schembri
+Schembra
+Schellhorn
+Schellenberge
+Schelle
+Scheitlin
+Scheidecker
+Scheibner
+Scheiblich
+Schehl
+Schefers
+Schee
+Schearer
+Schaubert
+Schattschneid
+Scharich
+Schares
+Scharber
+Schappach
+Schaneman
+Schamberger
+Schak
+Schaetzle
+Schaecher
+Scerbo
+Scelba
+Scavona
+Scatton
+Scarsdale
+Scarr
+Scarpone
+Scarlata
+Scariano
+Scandurra
+Scandura
+Scandalis
+Scammahorn
+Scafuto
+Scaffe
+Scachette
+Sayyed
+Sayko
+Sayco
+Sayasane
+Sayaphon
+Sawney
+Sawdo
+Sawatzke
+Sawallich
+Savko
+Savka
+Savitts
+Saviola
+Savio
+Savine
+Savich
+Savells
+Saulpaugh
+Saulino
+Sauler
+Saugis
+Sauber
+Sau
+Saturnio
+Sattel
+Satomba
+Saterfield
+Satava
+Sasseville
+Sasahara
+Sarzynski
+Sartorius
+Sartore
+Sartell
+Sarsour
+Sarson
+Sarp
+Sarnosky
+Sarni
+Sarlinas
+Sarka
+Sarinsky
+Sarin
+Sardo
+Sarden
+Sarchett
+Sarault
+Sarate
+Sarao
+Sarantakis
+Saralegui
+Sapper
+Sappah
+Sapinski
+Sapardanis
+Sapara
+Sanyaro
+Santwire
+Santrmire
+Santoriella
+Santor
+Santomassimo
+Santisteban
+Santillanez
+Santamarina
+Sansotta
+Sanpson
+Sannutti
+Sankoh
+Sangasy
+Sanfelix
+Sandvill
+Sandus
+Sandstede
+Sandling
+Sandland
+Sandhop
+Sandeen
+Sandblom
+Sanday
+Sandager
+Sancrant
+Sancken
+Sanchirico
+Sancher
+Sances
+Sanberg
+Sanacore
+Samyn
+Samul
+Samrov
+Samrah
+Sampere
+Sampang
+Samland
+Samii
+Samiento
+Sames
+Sambrook
+Samborski
+Samberg
+Samaroo
+Salzl
+Salvio
+Salvati
+Salvadge
+Saluan
+Saltzberg
+Saltus
+Saltman
+Salstrom
+Salotti
+Salmonsen
+Sallmen
+Salle
+Sallach
+Salines
+Salesky
+Saleme
+Saleha
+Saldano
+Salb
+Salazak
+Salasar
+Salado
+Salach
+Sakumoto
+Sakamaki
+Sajovic
+Sajous
+Sainte
+Sainliere
+Sainato
+Sails
+Saik
+Saieva
+Saice
+Sahe
+Sahady
+Sago
+Saft
+Safier
+Saffo
+Safer
+Saether
+Saens
+Saeler
+Saelens
+Sadvary
+Sadoski
+Sadorra
+Sadolsky
+Sadin
+Sadik
+Sadeghi
+Sadat
+Sacramed
+Sachetti
+Sacchi
+Sacca
+Saberi
+Saarela
+Saadat
+Saabatmand
+Rzeczycki
+Rysz
+Rynkowski
+Rynerson
+Ryneer
+Rymut
+Rymes
+Rymasz
+Rylaarsdam
+Rykaczewski
+Ryen
+Ryea
+Rydin
+Rydelek
+Rydel
+Rydeen
+Rybinski
+Ruvalcava
+Rutski
+Rutske
+Rutman
+Rutkin
+Ruths
+Ruthman
+Ruthers
+Rutheford
+Rutgers
+Rutenberg
+Rutar
+Russwurm
+Russomano
+Russomanno
+Russer
+Russello
+Rushanan
+Rusen
+Ruschmeyer
+Rusaw
+Rupnick
+Rupley
+Rupinski
+Ruopoli
+Rumps
+Rumbach
+Rulapaugh
+Ruivo
+Ruiter
+Ruhoff
+Ruhn
+Ruhman
+Ruggirello
+Ruffell
+Ruffel
+Ruezga
+Ruesga
+Ruelar
+Ruehter
+Ruehling
+Ruehlen
+Ruedas
+Rued
+Rueck
+Rudoy
+Rudio
+Rudh
+Rudell
+Rudat
+Rudack
+Ruckey
+Ruckel
+Ruckdaschel
+Rubsam
+Rubie
+Rubick
+Ruberti
+Rubeo
+Rubenfield
+Rubenfeld
+Rubash
+Rubalcave
+Rozzelle
+Rozon
+Royle
+Roxbury
+Rowlison
+Rowels
+Rowbotham
+Rovell
+Rouw
+Routzen
+Routzahn
+Routte
+Rousso
+Rousell
+Rous
+Rounsville
+Rouly
+Roulhac
+Roulette
+Roule
+Rouhoff
+Roughen
+Rouch
+Rottinghous
+Rottier
+Rotruck
+Rotkowski
+Rotkovecz
+Rothfeld
+Rotherham
+Rotch
+Rotanelli
+Rosul
+Rossie
+Rossen
+Rosseel
+Rosky
+Rosian
+Rosher
+Rosewall
+Roseum
+Roseth
+Rosenwinkel
+Rosentrater
+Rosenlof
+Rosenhagen
+Rosengren
+Rosendorf
+Rosendale
+Rosenbush
+Rosemore
+Rosek
+Rosebur
+Roscup
+Rosca
+Rosboril
+Rosazza
+Rosane
+Rorabacher
+Ropka
+Roofner
+Ronsini
+Ronnie
+Ronnfeldt
+Ronn
+Ronero
+Roner
+Ronayne
+Rona
+Ron
+Romprey
+Rommelfanger
+Romkema
+Romiro
+Romay
+Romanowicz
+Romanov
+Romanoff
+Romaniszyn
+Romanek
+Romane
+Rollf
+Rollag
+Rolfson
+Rolack
+Rokicki
+Rohrdanz
+Rohdenburg
+Rohal
+Rogowicz
+Rogish
+Rogian
+Rogens
+Rogado
+Roesslein
+Roesing
+Roerig
+Roenigk
+Roelle
+Roehler
+Rodvold
+Rodrigres
+Rodregues
+Rodolph
+Rodkin
+Rodiquez
+Rodina
+Rodero
+Roderman
+Roderiquez
+Rodenizer
+Rodenbough
+Rodebush
+Rodde
+Rocle
+Rochlitz
+Rochkes
+Rocheford
+Robyn
+Robusto
+Roberston
+Robbie
+Robbert
+Robberson
+Robair
+Roam
+Roadruck
+Roades
+Roaden
+Roadarmel
+Rizzardi
+Rivinius
+Riveras
+Rivello
+Rivelli
+Rivadulla
+Rittinger
+Rittie
+Rittichier
+Ritthaler
+Ritmiller
+Riskin
+Risien
+Rishor
+Risatti
+Ripson
+Ringold
+Ringen
+Rinfret
+Rineheart
+Rindal
+Rincan
+Rinauro
+Rinaldis
+Rina
+Rimkus
+Rimi
+Rimel
+Rimbach
+Rily
+Rillie
+Riller
+Rihner
+Riherd
+Rigley
+Rightmyer
+Righthouse
+Riggert
+Riggers
+Rigerman
+Rigas
+Rifai
+Riesner
+Rienzo
+Riemersma
+Riefer
+Ridgebear
+Rides
+Ridell
+Ridall
+Ricucci
+Ricley
+Rickerl
+Richemond
+Richelieu
+Richel
+Richardville
+Riccitelli
+Ricciardelli
+Ricardez
+Riblett
+Ribar
+Riase
+Rian
+Rhym
+Rhule
+Rhude
+Rhondes
+Rhodehamel
+Rhim
+Rheingold
+Rheaves
+Reznick
+Reynero
+Revolorio
+Revette
+Revelo
+Reuven
+Reusswig
+Reusser
+Reuhl
+Reuber
+Rettele
+Retka
+Retersdorf
+Resseguie
+Resper
+Resner
+Resides
+Reshard
+Resek
+Reseigh
+Repaci
+Renzullo
+Renuart
+Rentfrow
+Rennemeyer
+Renneker
+Renkes
+Renier
+Rendle
+Renburg
+Remsburg
+Remos
+Remmie
+Remmick
+Remlin
+Remkus
+Remfert
+Remey
+Remerez
+Remedies
+Remaly
+Relph
+Rellihan
+Relles
+Relaford
+Reksten
+Rekas
+Reitzes
+Reiten
+Reitema
+Reisin
+Reinmann
+Reinicke
+Reinholdt
+Reinheimer
+Reinfeld
+Reineman
+Reineking
+Reinartz
+Reimel
+Reik
+Reihe
+Reidling
+Reidler
+Reichenberg
+Reichenback
+Reho
+Rehnborg
+Rehnberg
+Rehart
+Regusters
+Regulus
+Reglin
+Reginal
+Reges
+Regensburg
+Regen
+Regas
+Reevers
+Reever
+Reeter
+Reedholm
+Redle
+Redic
+Redfear
+Reddekopp
+Rechel
+Rebick
+Rebholz
+Reazer
+Reauish
+Reath
+Reasinger
+Reas
+Reary
+Realmuto
+Reager
+Readenour
+Razze
+Rawicki
+Rawhoof
+Ravi
+Ravetti
+Ravenscraft
+Rava
+Rauf
+Rauelo
+Rattee
+Rattay
+Rattanachane
+Rattana
+Rathmanner
+Rathgeber
+Rathe
+Rathbum
+Rasul
+Rastogi
+Rastelli
+Rassman
+Rasmuson
+Rasely
+Raschko
+Raschilla
+Rasche
+Rasanen
+Rary
+Raring
+Raridon
+Rarey
+Raquel
+Rappenecker
+Rapelyea
+Ransier
+Ransberger
+Rannalli
+Ranjel
+Ranford
+Randoll
+Randklev
+Ramy
+Ramundo
+Ramu
+Ramsuer
+Ramstad
+Ramsbottom
+Ramphal
+Ramnarine
+Rammer
+Ramiscal
+Ramgel
+Ramesar
+Ramento
+Rambeau
+Ramales
+Ralon
+Rallison
+Rakich
+Raith
+Raiola
+Rainwaters
+Rainbott
+Raimundo
+Raimer
+Raimann
+Railing
+Rahl
+Rahama
+Ragusano
+Rafla
+Rafiq
+Rafi
+Raffone
+Raffo
+Rafail
+Raelson
+Raehl
+Raebel
+Radway
+Radue
+Radona
+Radisovich
+Radics
+Rademan
+Radeke
+Radder
+Radden
+Rackow
+Racitano
+Racina
+Rachar
+Racanello
+Rabuck
+Rabkin
+Rabidoux
+Rabello
+Rabel
+Rabara
+Qunnarath
+Quirindongo
+Quintel
+Quintano
+Quinlin
+Quinchia
+Quincel
+Quilling
+Quillian
+Quilliam
+Quillens
+Quihuiz
+Quiett
+Quicksall
+Quest
+Querta
+Querido
+Quent
+Quealy
+Quaye
+Quante
+Quamme
+Qualia
+Quaker
+Quagliano
+Quader
+Pytlewski
+Pyo
+Pylvainen
+Pyland
+Pych
+Py
+Puyear
+Puulei
+Puthiyamadam
+Putalavage
+Purzycki
+Purkerson
+Purcella
+Purce
+Puppe
+Pupa
+Pullon
+Pullie
+Pulgarin
+Pulford
+Pujals
+Puiatti
+Pugeda
+Puffett
+Puffenbarger
+Puertas
+Puddy
+Pucio
+Pucella
+Ptaszynski
+Psomiades
+Psencik
+Przybysz
+Przybycien
+Przedwiecki
+Pryzgoda
+Prvitt
+Pruskowski
+Prugh
+Prudent
+Prudden
+Provazek
+Protasewich
+Protain
+Proo
+Prondzinski
+Prokes
+Prohonic
+Progacz
+Proescher
+Prodan
+Privatsky
+Privateer
+Priore
+Prinzing
+Prinzi
+Printers
+Prigmore
+Priewe
+Prier
+Pribbeno
+Prezzia
+Preyor
+Prewer
+Prevett
+Preuitt
+Prepotente
+Prence
+Prekker
+Preisach
+Precythe
+Prebish
+Preato
+Prchlik
+Prazeres
+Prazak
+Prauner
+Prattella
+Prati
+Prat
+Prasser
+Prasomsack
+Praml
+Prabhakaran
+Prabel
+Poyneer
+Powroznik
+Powal
+Poux
+Poullion
+Pouliotte
+Pottier
+Potthast
+Potocnik
+Poties
+Poths
+Postuci
+Postal
+Posso
+Poser
+Portwine
+Portune
+Portaro
+Porrello
+Porreca
+Porrazzo
+Poremski
+Pore
+Porcello
+Popple
+Poppert
+Popowski
+Popovec
+Popke
+Popik
+Popielarczyk
+Popick
+Popi
+Poper
+Popelka
+Popec
+Poortinga
+Poorte
+Pooni
+Ponyah
+Pontin
+Pomerance
+Pomar
+Polynice
+Polyak
+Polverari
+Poltorak
+Polovoy
+Pollmann
+Pollio
+Pollinger
+Pollacco
+Polivka
+Polian
+Poleyestewa
+Polera
+Poldrack
+Polcovich
+Polakoff
+Polakis
+Poladian
+Pokorski
+Poiter
+Poffenroth
+Poetzsch
+Poeschl
+Poeschel
+Poepplein
+Poepping
+Poeling
+Podvin
+Podsiad
+Podrasky
+Podlas
+Pode
+Podbielski
+Podany
+Pochiba
+Pocchia
+Poalino
+Poaipuni
+Plymire
+Plyer
+Pluvoise
+Plungy
+Pluid
+Ploude
+Plosker
+Plomma
+Plohr
+Plocica
+Pliler
+Plevin
+Plessis
+Plesnarski
+Plesha
+Plenskofski
+Plecker
+Platenburg
+Platas
+Plansinis
+Plana
+Plamer
+Placencio
+Pizzolato
+Pizur
+Pius
+Piurkowski
+Pituch
+Pittillo
+Pitel
+Pitcak
+Piszczatowski
+Pisula
+Pishner
+Pirner
+Pirillo
+Pippert
+Pipe
+Pinyan
+Pinsonnault
+Pinnt
+Pinkelton
+Pinena
+Pinela
+Pineault
+Pinault
+Pilotti
+Pillips
+Pilbin
+Pilati
+Pikey
+Pih
+Piguet
+Pigna
+Pigler
+Pigat
+Pietzsch
+Pietrafesa
+Pieters
+Pierzchala
+Pierrie
+Pierfax
+Piercefield
+Piedmont
+Piedigrossi
+Piede
+Piechoski
+Piearcy
+Pidcock
+Picolet
+Pickren
+Pickings
+Picht
+Picco
+Pi
+Phomphithak
+Phommatheth
+Phlieger
+Phippen
+Philpotts
+Phillipi
+Philippon
+Philipose
+Philben
+Pherson
+Pherguson
+Phatdouang
+Phanthauong
+Phanord
+Pfirsch
+Pfendler
+Pfannenstein
+Pfahlert
+Pfahler
+Pezzuto
+Pezzimenti
+Pexton
+Pexsa
+Pewo
+Pevsner
+Petzel
+Petts
+Pettner
+Pettinella
+Petticrew
+Pettibon
+Pettes
+Petrov
+Petrosyan
+Petron
+Petrocelli
+Petrocco
+Petrizzo
+Petris
+Petrino
+Petricone
+Petralba
+Petrakis
+Petrain
+Petkoff
+Petitjean
+Petges
+Peteuil
+Petet
+Petersdorf
+Petchulis
+Pestronk
+Peskind
+Pesenti
+Pertsovsky
+Personette
+Persia
+Persampieri
+Persall
+Pers
+Perre
+Perper
+Perolta
+Perng
+Perler
+Perkoski
+Perish
+Perilloux
+Perey
+Peressini
+Percontino
+Perciballi
+Peral
+Peppas
+Pepitone
+Penzero
+Pentico
+Pent
+Penski
+Pense
+Penrice
+Penoyer
+Penovich
+Pennimpede
+Pennigton
+Pennig
+Penisson
+Pendl
+Pendill
+Penceal
+Penatac
+Penasa
+Penanegra
+Pelman
+Pelligrini
+Pelliccia
+Pellant
+Pelkowski
+Pelak
+Pein
+Peightell
+Pegler
+Pegelow
+Peffers
+Peetz
+Peelman
+Pee
+Pedrin
+Pedlow
+Pedelty
+Pede
+Peddy
+Peckinpaugh
+Peckens
+Pecht
+Pechin
+Peche
+Peccia
+Peca
+Peaker
+Pazik
+Pazderski
+Pazan
+Payno
+Payenda
+Pawluk
+Pawlosky
+Pawell
+Pavlikowski
+Pavlides
+Pavish
+Paviol
+Paulick
+Paukert
+Pattum
+Patrylak
+Patronella
+Patrich
+Patriarco
+Patraw
+Patierno
+Patient
+Patience
+Paten
+Pastorin
+Pasternack
+Pastano
+Passaro
+Pasqualino
+Paskoff
+Paskin
+Paskiewicz
+Pashel
+Pasey
+Pascher
+Pasaye
+Pasanen
+Parvis
+Partmann
+Parthemore
+Parshotam
+Parsens
+Parraga
+Paronto
+Paroda
+Parobek
+Parmann
+Parmalee
+Parlet
+Parle
+Parkers
+Pariente
+Paree
+Pardey
+Parde
+Pardall
+Parbs
+Parbol
+Paranada
+Parah
+Parado
+Pappy
+Pappenheim
+Paplow
+Papka
+Papich
+Papi
+Papallo
+Paolicelli
+Panzarella
+Panyik
+Pantle
+Pantera
+Pantalone
+Pansullo
+Panone
+Pano
+Panny
+Pannenbacker
+Pankiewicz
+Pankhurst
+Panke
+Pankau
+Pangan
+Panessa
+Pandolfi
+Pandiani
+Panchik
+Panchak
+Panakos
+Panak
+Panagakos
+Palubiak
+Palso
+Palowoda
+Palmucci
+Palmour
+Palmino
+Palmerino
+Palme
+Pallino
+Pallerino
+Palisi
+Palisano
+Palis
+Palazzola
+Palay
+Palaspas
+Palamara
+Paladini
+Paladin
+Paire
+Paillet
+Pailet
+Paider
+Paguin
+Pagoda
+Paglione
+Paglialunga
+Pageau
+Pagdanganan
+Pafundi
+Padiong
+Padberg
+Padarebones
+Padalecki
+Pacol
+Pacilio
+Pachter
+Pachew
+Pabelick
+Paaske
+Ozzella
+Owoc
+Owca
+Ovitz
+Overmann
+Overlee
+Overhulser
+Overholtzer
+Ovens
+Ovall
+Outhier
+Ouren
+Ouinones
+Ottum
+Ottomaniello
+Otteman
+Otsman
+Otinger
+Oszust
+Ostorga
+Ostolaza
+Osterhouse
+Osterberger
+Ostberg
+Ososki
+Osmers
+Osmera
+Oshey
+Osequera
+Osenkowski
+Oschmann
+Osbment
+Osbey
+Osazuwa
+Osayande
+Osako
+Orzell
+Orvin
+Ortwine
+Ortmeyer
+Ortelt
+Ortelli
+Orsten
+Orson
+Orrill
+Orphey
+Orndorf
+Orloski
+Orlich
+Orlander
+Orland
+Ork
+Orji
+Orison
+Orielly
+Orielley
+Ori
+Organek
+Orey
+Orender
+Ordona
+Ordon
+Ordman
+Orazine
+Oravetz
+Orandello
+Orabone
+Ora
+Or
+Oquenda
+Opyd
+Opteyndt
+Opoka
+Opiola
+Opielski
+Opell
+Opeka
+Onyeagu
+Onezne
+Ondeck
+Ona
+Oms
+Ommen
+Ominelli
+Omernik
+Omelia
+Olynger
+Olwin
+Olvey
+Olufson
+Olubunmi
+Olten
+Olshefski
+Olsby
+Olores
+Olma
+Olli
+Ollech
+Ollar
+Oliviera
+Olivarri
+Oligschlaeger
+Olheiser
+Olgin
+Olevera
+Olerud
+Olenski
+Olenius
+Oldow
+Oldershaw
+Oldenburger
+Olausen
+Olaes
+Okutsu
+Okken
+Okitsu
+Okie
+Okeson
+Okelberry
+Okel
+Ojito
+Ojano
+Ohyama
+Ohr
+Ohnstad
+Ohmen
+Ohlhauser
+Ohlensehlen
+Ohle
+Ohashi
+Ohanley
+Ogzewalla
+Ogutu
+Ogston
+Ogrodowicz
+Oginski
+Ogiamien
+Oger
+Ogarro
+Ofsak
+Oflynn
+Off
+Ofer
+Oelze
+Oehm
+Oehlschlager
+Oehl
+Odome
+Odo
+Odmark
+Odil
+Odgen
+Odermott
+Odair
+Oczon
+Ockman
+Ockleberry
+Ocken
+Ochal
+Ochakovsky
+Ocenasek
+Occhuizzo
+Ocanaz
+Obrein
+Obray
+Oborne
+Oblinski
+Obin
+Obierne
+Obholz
+Obhof
+Oberski
+Obermier
+Oberlies
+Obergfell
+Obenauer
+Obeid
+Obbink
+Obaker
+Oatney
+Oatfield
+Nyulassy
+Nwagbara
+Nutley
+Nuth
+Nurthen
+Nuntaray
+Nunno
+Nunlee
+Nuner
+Numkena
+Nuhfer
+Nugal
+Nuessen
+Nuding
+Nuchols
+Noye
+Noya
+Nowosielski
+Novickis
+Novi
+Novencido
+Novel
+Novad
+Noujaim
+Notoma
+Notice
+Noth
+Notch
+Notarnicola
+Nosworthy
+Nosacka
+Norum
+Northouse
+Nortesano
+Norstrand
+Norsingle
+Norrie
+Norr
+Norn
+Normoyle
+Norise
+Nordstrand
+Nordmark
+Nordes
+Norales
+Nopachai
+Noorda
+Nooman
+Nonroe
+Nonemaker
+Nonamaker
+Nommay
+Noman
+Nollet
+Nolle
+Noli
+Noice
+Noerr
+Nodland
+Nocon
+Nocks
+Nockels
+Nocella
+Nocek
+Njie
+Nizo
+Nitchman
+Nistendirk
+Nissan
+Nisly
+Nishitani
+Nishio
+Nishina
+Nirschl
+Niro
+Nirenberg
+Niquette
+Nip
+Nindorf
+Nincehelsor
+Nimz
+Nimura
+Nilmeier
+Nikula
+Nikach
+Nik
+Nightwine
+Night
+Nighman
+Nighbor
+Niffenegger
+Niez
+Niesporek
+Nier
+Nieminen
+Niemie
+Niedermeier
+Niederberger
+Nido
+Nicome
+Nicolozakes
+Nicolia
+Nicoles
+Nicolau
+Nickodem
+Nicklous
+Nickisch
+Nicka
+Nici
+Nibler
+Nibbe
+Nhatsavang
+Ngoun
+Neyer
+Newmyer
+Newitt
+Newgard
+Newenle
+Newbraugh
+Newbound
+Newand
+Nevue
+Nevison
+Nevis
+Nev
+Neujahr
+Neufer
+Nette
+Netkowicz
+Nethkin
+Nesvig
+Nestico
+Nessner
+Nesslein
+Nesset
+Nessel
+Neshem
+Nesbeth
+Neris
+Nerenberg
+Neren
+Nepomuceno
+Nemith
+Nelder
+Neitzke
+Neita
+Neiner
+Neimeyer
+Neigenfind
+Neiford
+Neidenbach
+Nehlsen
+Negreta
+Negrana
+Neenan
+Neddenriep
+Nech
+Neborak
+Nebesny
+Nazar
+Nawfel
+Navo
+Navarete
+Nauss
+Naumes
+Naugler
+Nauer
+Natvig
+Natalizio
+Natalie
+Natalia
+Nastasia
+Nasaire
+Naruaez
+Narrow
+Narkevicius
+Nardozzi
+Nardino
+Narain
+Napue
+Napenas
+Nap
+Naomi
+Nao
+Nanz
+Nantwi
+Nannen
+Nang
+Nanfito
+Nanes
+Nan
+Namsaly
+Namey
+Namer
+Namauu
+Namanworth
+Nalevanko
+Nalder
+Nakaoka
+Nakamatsu
+Nakajima
+Nakada
+Nakaahiki
+Naimoli
+Nahmias
+Nahhas
+Nagtalon
+Nagelkirk
+Nagasawa
+Naftel
+Nadine
+Naderman
+Nachbar
+Nacci
+Nabzdyk
+Nabor
+Nabavian
+Nabarowsky
+Naasz
+Myslim
+Myree
+Mylar
+Myall
+Muzii
+Muyres
+Muwwakkil
+Mutters
+Mutschelknaus
+Musulin
+Mustaro
+Mustache
+Musslewhite
+Mussell
+Mussa
+Musni
+Muslim
+Muskrat
+Muskopf
+Muskett
+Musitano
+Musilli
+Musielak
+Musguire
+Musgraves
+Muscott
+Muschik
+Muschaweck
+Mursch
+Murril
+Murra
+Muros
+Muri
+Murel
+Murcko
+Murak
+Muphy
+Muntean
+Mundz
+Mundinger
+Munder
+Mumaugh
+Mulville
+Mulrenin
+Mulnix
+Mullenaux
+Mullahy
+Mulkern
+Mulkerin
+Mulchrone
+Mulato
+Muinos
+Muhlstein
+Mugnolo
+Muggeo
+Mugge
+Muffett
+Muenzenberger
+Muellerleile
+Mudie
+Muckelroy
+Muccio
+Mrvan
+Mrkvicka
+Mraw
+Mozick
+Mozga
+Mozak
+Moxness
+Moxey
+Mounkes
+Mound
+Motonaga
+Mothershead
+Motayne
+Motayen
+Mosty
+Mostad
+Mossbarger
+Moskwa
+Moskop
+Mosena
+Mosen
+Moscoffian
+Moryl
+Morvillo
+Mortin
+Mortier
+Morsberger
+Morrey
+Morrales
+Morral
+Morphy
+Morock
+Morlino
+Morkert
+Morken
+Morisseau
+Morishito
+Morinville
+Morici
+Morgano
+Morgana
+Moreschi
+Morenco
+Morence
+Morella
+Mordeci
+Moratto
+Morath
+Morario
+Morando
+Moradian
+Morada
+Mootry
+Moomey
+Monville
+Montoto
+Montore
+Montoney
+Montfort
+Montey
+Montesi
+Monterrubio
+Montembeau
+Montayes
+Montalban
+Montaivo
+Monsay
+Monot
+Monopoli
+Monnerjahn
+Monkowski
+Monka
+Monjure
+Monios
+Monington
+Monges
+Monfils
+Moneyhun
+Moneaux
+Mondt
+Mondoza
+Mondloch
+Mondelli
+Mondale
+Monclova
+Moncher
+Monath
+Monagas
+Mominee
+Moma
+Molz
+Molstad
+Molsan
+Molnau
+Mollura
+Molleur
+Molla
+Molands
+Moitoza
+Moisa
+Moine
+Mohrlock
+Mohre
+Mohomed
+Mohmed
+Mohair
+Mogus
+Moeuy
+Moeser
+Moehr
+Moehle
+Modique
+Modgling
+Modglin
+Moderski
+Moczulski
+Moccasin
+Moayyad
+Moatz
+Mlodzianowski
+Mleczynski
+Mizwicki
+Mizutani
+Mizia
+Mizenko
+Miyataki
+Miyanaga
+Miville
+Mitsdarffer
+Mitrani
+Mitman
+Mitkowski
+Misuraca
+Miskinis
+Miskiewicz
+Miska
+Misik
+Mishulovin
+Mishulouin
+Mishkin
+Mishar
+Misenti
+Mischo
+Mischnick
+Mirisola
+Miricle
+Mirick
+Miramontez
+Mirafuentes
+Miraflores
+Miquel
+Mione
+Minzy
+Minzenmayer
+Minzenberger
+Mintken
+Minten
+Minot
+Minors
+Minn
+Minkowitz
+Minkins
+Minister
+Minic
+Minhas
+Mingioni
+Mingee
+Minert
+Minchow
+Mincer
+Minalga
+Mimozo
+Milward
+Milson
+Milosch
+Millings
+Millick
+Millare
+Milke
+Milinazzo
+Milin
+Milich
+Milette
+Mile
+Mildrum
+Mildon
+Milcher
+Milberger
+Mikuszewski
+Miklitz
+Mikko
+Mihalios
+Mihalick
+Mieth
+Mierzwiak
+Mierzwa
+Mierow
+Mierez
+Mierau
+Mielcarek
+Miecznikowski
+Miears
+Middlekauff
+Micucci
+Mickelberry
+Michno
+Michlich
+Michieli
+Michelstein
+Michelini
+Michalicek
+Michal
+Micciche
+Micalizzi
+Mguyen
+Mezzina
+Mezzenga
+Meydid
+Meusel
+Meusa
+Metty
+Mettig
+Mettenburg
+Metier
+Meth
+Metelko
+Mestemacher
+Messamore
+Mesplay
+Mespelt
+Mesiti
+Mesina
+Meshyock
+Mesenbring
+Meschke
+Merzlak
+Merrih
+Merner
+Merkwan
+Merklein
+Merkey
+Meringolo
+Merine
+Mergist
+Merganthaler
+Merckling
+Menzer
+Mensalvas
+Mennecke
+Menne
+Menjiva
+Mengwasser
+Menger
+Menedez
+Meneal
+Menck
+Mencia
+Menchen
+Menchavez
+Melzer
+Melve
+Melso
+Meloan
+Melman
+Mellison
+Mellerson
+Mellendorf
+Mellberg
+Melikian
+Melian
+Melgaard
+Meleo
+Melbye
+Melber
+Meja
+Meixelberger
+Meitz
+Meitner
+Meiss
+Meisch
+Meinen
+Meinberg
+Meigel
+Meierhofer
+Mehringer
+Mehrer
+Mehle
+Mehall
+Megahan
+Mega
+Mefferd
+Meenan
+Meecham
+Medvec
+Medinger
+Meddock
+Medawar
+Medaries
+Mecias
+Mecannic
+Meazell
+Measom
+Meaden
+Meach
+Mcwhinnie
+Mcwhinney
+Mcwells
+Mcvinney
+Mcvenes
+Mcthige
+Mcthay
+Mcshaw
+Mcroyal
+Mcrenolds
+Mcratt
+Mcquilliams
+Mcquesten
+Mcphetridge
+Mconnell
+Mcnolty
+Mcneish
+Mcnany
+Mcnamar
+Mcmullins
+Mcmulen
+Mcmenimen
+Mcmellen
+Mcmanuis
+Mcmanemy
+Mclernon
+Mclauren
+Mclamore
+Mckusick
+Mckosky
+Mckirryher
+Mckindra
+Mckin
+Mckever
+Mckernin
+Mckerlie
+Mckennzie
+Mckelvin
+Mckelphin
+Mckeague
+Mckaughan
+Mciwraith
+Mcilhinney
+Mchardy
+Mcgurie
+Mcgrevey
+Mcgreen
+Mcgohan
+Mcglocklin
+Mcglew
+Mcglaun
+Mcgibney
+Mcghinnis
+Mcgaughan
+Mcgathy
+Mcferran
+Mcfeely
+Mcfatten
+Mcewin
+Mcendarfer
+Mcenany
+Mcelvy
+Mcelmarry
+Mceathron
+Mceaddy
+Mcdugle
+Mcdoulett
+Mcdaneld
+Mcculloh
+Mccullin
+Mccullan
+Mccullagh
+Mccubrey
+Mccrobie
+Mccrain
+Mccraight
+Mccracker
+Mccrabb
+Mccowin
+Mccoubrey
+Mccoon
+Mcconomy
+Mcconnico
+Mcconahay
+Mccomish
+Mccoid
+Mccloude
+Mcclinsey
+Mcclenic
+Mcclee
+Mccier
+Mccathran
+Mccash
+Mccarvy
+Mccarrol
+Mccarraher
+Mccalpane
+Mccalebb
+Mccalanahan
+Mccade
+Mccadams
+Mcbroome
+Mcaskill
+Mcartor
+Mcaree
+Mbonu
+Mazzillo
+Mazzetti
+Mazuera
+Mazowieski
+Mazierski
+Mazella
+Mayze
+Maywalt
+Mayher
+Mawk
+Mavris
+Maushardt
+Mauras
+Mauracher
+Maupins
+Matysiak
+Matye
+Matusz
+Matuska
+Matusiewicz
+Matulewicz
+Mattock
+Mattingley
+Mattina
+Mattick
+Mattan
+Matskin
+Matros
+Matrisciano
+Matone
+Matonak
+Matlow
+Matkovic
+Matison
+Mathelier
+Matelski
+Mateiro
+Masunaga
+Masterton
+Mastalski
+Massini
+Massena
+Massed
+Massarelli
+Massanelli
+Maso
+Maslen
+Maslakowski
+Masincup
+Masilko
+Masher
+Mashall
+Masello
+Masell
+Maschmeyer
+Mascheck
+Maschak
+Mascari
+Masar
+Masak
+Masaitis
+Marxsen
+Maruschak
+Maruscak
+Marus
+Marumoto
+Martyr
+Martsolf
+Martorelli
+Martling
+Martischnig
+Martirano
+Martinsons
+Martinov
+Martinon
+Martinolli
+Martinet
+Martinell
+Martinel
+Martinat
+Martich
+Martey
+Martelles
+Martelle
+Marsolais
+Marsili
+Marshbanks
+Marshak
+Marseilles
+Marsaw
+Marrier
+Marrett
+Marrapodi
+Marrapese
+Marquitz
+Marousek
+Maronge
+Maro
+Marmerchant
+Marlene
+Markworth
+Markwardt
+Markuson
+Markou
+Markakis
+Marjenhoff
+Maritato
+Mariska
+Mariacher
+Margot
+Margis
+Marflak
+Marfil
+Marer
+Mardirossian
+Marcusen
+Marconis
+Marcisak
+Marcille
+Marchionni
+Marchesi
+Marchaland
+Marcet
+Marcelli
+Marca
+Marbley
+Marash
+Marascalco
+Marante
+Marangoni
+Marando
+Mapua
+Mapstone
+Mapa
+Maohu
+Manzur
+Manweiler
+Manuia
+Manto
+Mantifel
+Mantia
+Manteuffel
+Mantella
+Manteca
+Manspeaker
+Mansbach
+Manous
+Manoso
+Manolis
+Manocchia
+Mannheim
+Mannello
+Manlangit
+Manino
+Manieri
+Manicchio
+Maniar
+Maniaci
+Maniace
+Manglona
+Mangis
+Mangiafico
+Manghane
+Manero
+Manely
+Maneafaiga
+Mandril
+Mandolfo
+Mander
+Mandelberg
+Mandala
+Manco
+Mancill
+Mancher
+Manche
+Manaugh
+Manassa
+Manasares
+Manansala
+Manalili
+Mamudoski
+Mammo
+Mammenga
+Mamaril
+Mamaclay
+Malueg
+Malter
+Maltbia
+Maltas
+Malool
+Mallas
+Mallalieu
+Mallacara
+Malkiewicz
+Malinovsky
+Malewski
+Malett
+Maldomado
+Malcomson
+Malcik
+Malavet
+Malaver
+Malasky
+Malas
+Malango
+Malanaphy
+Malach
+Makofsky
+Mako
+Makler
+Maka
+Majuste
+Majied
+Majeske
+Majerowski
+Majera
+Maixner
+Maisto
+Maiocco
+Mailo
+Maile
+Maikoksoong
+Mahunik
+Mahrer
+Mahraun
+Maholmes
+Mahlke
+Mahli
+Mahfouz
+Maheia
+Mahalko
+Magwire
+Magpuri
+Magoun
+Magnone
+Magnetti
+Magliulo
+Magliolo
+Magliocco
+Magitt
+Magginson
+Maggert
+Magera
+Maged
+Mage
+Magbitang
+Magalong
+Magaha
+Maffitt
+Maffey
+Maestri
+Maenpaa
+Maenhout
+Maendel
+Mady
+Maduro
+Madu
+Madray
+Madras
+Madock
+Madlung
+Madler
+Madenford
+Madeau
+Maddaleno
+Macvean
+Macura
+Macrum
+Macrostie
+Macnaught
+Macnamee
+Macmurray
+Macmillen
+Maclay
+Mackle
+Mackimmie
+Mackedanz
+Maciejko
+Maciasz
+Maciak
+Machtley
+Machens
+Macentee
+Maceda
+Macdougald
+Maccauley
+Maccartney
+Macareno
+Macaraig
+Macapagal
+Macahilas
+Macadamia
+Mabone
+Mabary
+Maatta
+Maalouf
+Lysak
+Lynge
+Lynady
+Lykam
+Lyerla
+Lychwala
+Luzuriaga
+Luzinski
+Luxon
+Luvene
+Lutzi
+Luthe
+Luss
+Lushbaugh
+Luscavage
+Lurey
+Luquin
+Lupul
+Lupu
+Lupkin
+Lupfer
+Luoto
+Lundman
+Lundie
+Lundi
+Lundemo
+Luncsford
+Lumukanda
+Lumpp
+Lummis
+Lumantas
+Luloff
+Lukavsky
+Luitjens
+Luhring
+Luga
+Luffy
+Luelf
+Luehring
+Luedi
+Lueckenotte
+Luecht
+Luebano
+Ludvik
+Ludovici
+Ludkowski
+Luderman
+Luddy
+Lucksom
+Luckritz
+Luckadoo
+Lucion
+Luci
+Luchessa
+Luchesi
+Lucear
+Lucario
+Luben
+Luangsingotha
+Lozzi
+Lozo
+Loyst
+Loyed
+Lowin
+Lowber
+Lovich
+Lovenbury
+Loveh
+Lovec
+Louser
+Louris
+Lourence
+Loureiro
+Louras
+Lounds
+Loukidis
+Loukas
+Louissant
+Louer
+Louch
+Lotze
+Lotthammer
+Lotter
+Loterbauer
+Lotempio
+Lostracco
+Loston
+Lossman
+Loson
+Loskill
+Loske
+Loshe
+Lorz
+Lorion
+Lopuzzo
+Lopilato
+Lopera
+Loosey
+Looi
+Loock
+Lonsway
+Lons
+Longueville
+Longton
+Longknife
+Longin
+Longfield
+Longcor
+Londner
+Lompa
+Lommel
+Lomg
+Lolling
+Lolli
+Loli
+Lolar
+Lokuta
+Lokke
+Lokhmator
+Lojek
+Lois
+Loil
+Lohmeier
+Logero
+Loewe
+Loessberg
+Loeschner
+Loesche
+Loehlein
+Loeckle
+Loebs
+Loduca
+Lodense
+Lodeiro
+Locsin
+Locorriere
+Locklier
+Lockette
+Lochotzki
+Loche
+Locantore
+Locante
+Lobosco
+Lobingier
+Loats
+Loarca
+Llyod
+Llopis
+Llarenas
+Ljungquist
+Lizer
+Lizarda
+Livi
+Livezey
+Liverani
+Livas
+Liuzza
+Litzsinger
+Litza
+Littlehale
+Litter
+Litehiser
+Litecky
+Liskovec
+Liskiewicz
+Liskai
+Lisius
+Lisiecki
+Lisherness
+Lisanti
+Lipstone
+Lipsitz
+Lippi
+Lipovsky
+Lipkind
+Lipke
+Lipitz
+Lipa
+Liontos
+Linzie
+Linstrom
+Linssen
+Linsner
+Linsay
+Linnecke
+Linnan
+Linkkila
+Linginfelter
+Lingberg
+Lingardo
+Lingao
+Linea
+Lindwall
+Lindskog
+Lindline
+Lindesmith
+Lincicum
+Linahan
+Limthong
+Limesand
+Limauro
+Limardo
+Lilleberg
+Liljedahl
+Liljeberg
+Lilja
+Likio
+Ligons
+Lifshitz
+Liesch
+Lierle
+Lienke
+Lienemann
+Liekhus
+Liederbach
+Lieder
+Liechti
+Liebskind
+Liebhardt
+Liebelt
+Lie
+Liddie
+Lidbom
+Licor
+Lico
+Lickness
+Lickiss
+Lickey
+Lichtig
+Lichtenwalter
+Lichte
+Lichstein
+Lichorat
+Lichlyter
+Liccione
+Licalzi
+Librizzi
+Libre
+Librandi
+Libke
+Libert
+Liano
+Lianes
+Lezon
+Lezer
+Lezak
+Leynes
+Lewton
+Lewry
+Lewandowsky
+Levo
+Levites
+Levitch
+Levitas
+Levister
+Levinsky
+Leverentz
+Levendosky
+Leuty
+Leuters
+Leusink
+Leupold
+Leuchs
+Letteney
+Letteer
+Letrent
+Letourneaux
+Letofsky
+Letman
+Letko
+Letang
+Letalien
+Lestelle
+Lessin
+Lessenberry
+Lessen
+Lessa
+Lespier
+Lesky
+Leshure
+Leshko
+Lescavage
+Lermond
+Lerew
+Leonti
+Leonaggeo
+Lenza
+Lenters
+Lenord
+Lenny
+Lennert
+Lenix
+Lening
+Lengle
+Lengacher
+Lener
+Leneave
+Lencioni
+Lempe
+Lemone
+Lemin
+Lemich
+Lemert
+Lelis
+Lele
+Lekwa
+Lejune
+Leitze
+Leitem
+Leistner
+Leipheimer
+Leimkuehler
+Leiding
+Leidel
+Leidall
+Leichty
+Leichtman
+Leibenstein
+Leiba
+Lehrian
+Lehrfeld
+Legrow
+Legrant
+Legore
+Leghorn
+Legel
+Legallo
+Lefew
+Leemow
+Leebrick
+Ledy
+Leduke
+Ledon
+Ledley
+Ledec
+Ledebuhr
+Lecoultre
+Leconey
+Leckington
+Lechlak
+Lechel
+Lebovic
+Lebourgeois
+Leberman
+Lebario
+Leavelle
+Leasy
+Leah
+Leagjeld
+Leafe
+Leabow
+Lazzar
+Lazer
+Lazenson
+Lazenberry
+Layher
+Lawe
+Lavon
+Lavina
+Lavette
+Laverne
+Laverette
+Lavee
+Lavear
+Lavatch
+Lauwers
+Lauw
+Lauture
+Lautman
+Lauters
+Laurion
+Laurens
+Laurenceau
+Launt
+Launelez
+Laughbaum
+Lauerman
+Laudat
+Laubacher
+Latzka
+Latzig
+Latortue
+Lathon
+Lathim
+Latessa
+Latella
+Lataille
+Lasyone
+Lastovica
+Lasselle
+Lask
+Lashutva
+Laserna
+Lascody
+Lasaint
+Larve
+Laruffa
+Larsh
+Larreta
+Larko
+Largay
+Larey
+Lardydell
+Larde
+Laravie
+Larate
+Laquay
+Lapuz
+Laprairie
+Lapora
+Lapiana
+Lanzoni
+Lanzillotti
+Lanzillo
+Lanzer
+Lanzalotti
+Lanton
+Lantey
+Lansdowne
+Lansden
+Lansang
+Lanquist
+Lanosga
+Lanosa
+Laninga
+Langsdale
+Langoni
+Langlands
+Langhout
+Langhorst
+Langenheim
+Langehennig
+Laneve
+Landucci
+Landsberry
+Landrey
+Landolfo
+Landkamer
+Landham
+Landgrebe
+Landefeld
+Lampp
+Lamparski
+Lamorgese
+Lamorella
+Lammie
+Lamielle
+Lamela
+Lambourne
+Lambino
+Lamberto
+Lamber
+Lambeck
+Lamascolo
+Lamarsh
+Lamantagne
+Lamaitre
+Lalumiere
+Lallo
+Laliberty
+Lalata
+Lalanne
+Laland
+Lakner
+Laity
+Lahrman
+Lahmann
+Lahip
+Lagroon
+Lagoa
+Laginess
+Lagge
+Lagatella
+Lagassie
+Laganga
+Lafranca
+Lafosse
+Laffredo
+Laferty
+Lafera
+Lafaver
+Lafauci
+Laesser
+Ladyman
+Ladtkow
+Laditka
+Ladeau
+Ladas
+Lacouette
+Lacosta
+Lacock
+Lacks
+Lackman
+Lackie
+Lachley
+Lacassagne
+Labrune
+Labrode
+Labreque
+Labrec
+Labog
+Labkovsky
+Labita
+Labbie
+Lababit
+Laaker
+Kylish
+Kyhn
+Kwiat
+Kwasny
+Kwack
+Kvilhaug
+Kuznicki
+Kuzmish
+Kuzmanic
+Kuzemchak
+Kuttler
+Kutella
+Kutchin
+Kuszlyk
+Kusumoto
+Kusuma
+Kustes
+Kusinski
+Kushlan
+Kushiner
+Kushin
+Kusak
+Kurzyniec
+Kury
+Kurter
+Kurrie
+Kurpiel
+Kurkjian
+Kurk
+Kurisu
+Kupres
+Kuokkanen
+Kunzie
+Kunzel
+Kunis
+Kuning
+Kundrick
+Kundla
+Kundinger
+Kully
+Kullas
+Kulkarni
+Kulcona
+Kulak
+Kulacz
+Kuks
+Kuklis
+Kuka
+Kuja
+Kuizinas
+Kuhtz
+Kuhnle
+Kuhnen
+Kuhnemund
+Kuhnel
+Kuhens
+Kuharik
+Kufner
+Kufeldt
+Kuenstler
+Kuehnert
+Kudzma
+Kudasik
+Kuczkowski
+Kucinskas
+Kuchto
+Kuch
+Kucel
+Kucek
+Kubica
+Kubecka
+Kuban
+Kszaszcz
+Krzywicki
+Krzynowek
+Krzal
+Krystal
+Krysiak
+Krys
+Krutsch
+Kruss
+Krusen
+Krusemark
+Krupiak
+Krumsiek
+Kruml
+Krulish
+Krulik
+Krulicki
+Krueth
+Kruer
+Kruel
+Krows
+Krossen
+Krolikowski
+Krolczyk
+Kroetch
+Kriticos
+Krites
+Krisher
+Krinke
+Krienke
+Kriegh
+Krichbaum
+Kribbs
+Kretchmar
+Kreitzbender
+Kreitler
+Kreinbring
+Kreb
+Kreamalmeyer
+Kreager
+Krawiecz
+Krawetz
+Krasley
+Krapfl
+Kranze
+Kranendonk
+Kramper
+Krampe
+Kramm
+Kralicek
+Krajnovich
+Krajcer
+Krain
+Kracker
+Kozinski
+Kownacki
+Kown
+Kowing
+Kowallis
+Kowall
+Kowalcyk
+Kowalchick
+Kovacic
+Kourt
+Kourkoumellis
+Kounter
+Kounlavong
+Kounce
+Koulabout
+Koualeski
+Kotzur
+Kottsick
+Kottre
+Kotte
+Kotrys
+Kotow
+Kothenbeutel
+Kotara
+Kostyla
+Kostich
+Kostenko
+Kossmann
+Kossin
+Kossakowski
+Kossack
+Kosoff
+Kosmatka
+Koshiol
+Koscielak
+Koscho
+Korzenski
+Kortz
+Kortum
+Korthauer
+Korshak
+Korsen
+Korol
+Korns
+Kornprobst
+Kornman
+Kormann
+Korineck
+Korf
+Koretsky
+Korenic
+Korbal
+Koralewski
+Koppelmann
+Kopis
+Kopiak
+Kopera
+Kopchick
+Kooken
+Kontogianis
+Konon
+Konn
+Konieczko
+Konick
+Konicek
+Koneval
+Kondratowicz
+Koncan
+Konat
+Komsthoeft
+Komosinski
+Kommer
+Kominek
+Koman
+Kolthoff
+Kology
+Kolnik
+Kolmetz
+Kolling
+Kolkowski
+Kolkemeyer
+Kolias
+Kolen
+Kolehmainen
+Kolby
+Kolberg
+Kolat
+Kokoska
+Koistinen
+Kohnert
+Kohlmyer
+Kofutua
+Kofoid
+Kofler
+Kofa
+Koetz
+Koetje
+Koerper
+Koeppl
+Koenning
+Koenigstein
+Koenigsfeld
+Koelle
+Koegel
+Koebley
+Koczera
+Kochmanski
+Kocaj
+Koc
+Koblick
+Kobis
+Kobialka
+Kobernick
+Kobak
+Knost
+Knori
+Knopinski
+Knoepfler
+Knoche
+Knipping
+Knipfel
+Knighter
+Kniefel
+Knie
+Knickman
+Knezevic
+Knewtson
+Knestrick
+Knesel
+Kneifel
+Knavel
+Knappe
+Knackstedt
+Klusmeyer
+Klus
+Klund
+Klun
+Kloos
+Kloock
+Kloiber
+Klohr
+Kloepper
+Klocek
+Klis
+Klingerman
+Klingen
+Klines
+Klimkowicz
+Kliever
+Kliem
+Kleypas
+Klevene
+Kleppinger
+Kleparek
+Klepacz
+Klemenc
+Klemanski
+Kleinwolterin
+Kleinsmith
+Kleinke
+Kleinberger
+Kleidon
+Kleespies
+Kleese
+Kleekamp
+Kleban
+Klayman
+Klay
+Klaver
+Klarman
+Klarberg
+Klapperich
+Kjetland
+Kizewski
+Kiyabu
+Kivioja
+Kittner
+Kittelberger
+Kissik
+Kisser
+Kishaba
+Kisch
+Kirner
+Kirkpatric
+Kirchhofer
+Kirchgessner
+Kirchausen
+Kirbie
+Kiral
+Kippes
+Kipper
+Kippel
+Kintsel
+Kintop
+Kinseth
+Kinroth
+Kinnion
+Kinningham
+Kinnier
+Kinnie
+Kinkin
+Kinkella
+Kingshott
+Kingore
+Kingen
+Kinerson
+Kindermann
+Kinart
+Kinan
+Kinabrew
+Kimbral
+Killean
+Kilcrest
+Kilb
+Kilarjian
+Kiffe
+Kientz
+Kiening
+Kielich
+Kieger
+Kieft
+Kieff
+Kiefel
+Kie
+Khum
+Khu
+Khov
+Khounborine
+Khoun
+Khoo
+Khensovan
+Khela
+Khay
+Khansari
+Khanponaphan
+Khano
+Khammixay
+Khalife
+Khalifah
+Khachatoorian
+Keyna
+Kexel
+Kewish
+Kettmann
+Ketring
+Ketler
+Ketcheside
+Ket
+Kestle
+Kessner
+Kerzer
+Kerss
+Kerska
+Kershbaumer
+Keros
+Kerntke
+Kerkel
+Keri
+Kerger
+Kereluk
+Kerechanko
+Kercado
+Keppers
+Keohane
+Kennet
+Kennealy
+Kenely
+Keneally
+Kendrew
+Kenderdine
+Kenagy
+Kenady
+Kemner
+Kemmler
+Kemme
+Kemerer
+Kelzer
+Kellon
+Kello
+Kellin
+Kellebrew
+Kellaway
+Keliipio
+Kelder
+Kelash
+Keitzer
+Keigley
+Keicher
+Kegerries
+Keens
+Keemer
+Keckler
+Keaveny
+Keath
+Keasley
+Kears
+Keany
+Keanum
+Keamo
+Kealohanui
+Kazmi
+Kazmer
+Kazin
+Kazeck
+Kazakos
+Kayrouz
+Kaylo
+Kawata
+Kaveny
+Kavadias
+Kauphusman
+Kaune
+Kaull
+Kaub
+Katzberg
+Katynski
+Katula
+Katten
+Katsbulas
+Katnik
+Katechis
+Katcsmorak
+Katan
+Kastning
+Kastman
+Kassell
+Kassabaum
+Kasprak
+Kasica
+Kasack
+Karvonen
+Karvis
+Karpowich
+Karpiak
+Karnish
+Karma
+Karell
+Kareem
+Kardashian
+Karczewski
+Karayan
+Karatz
+Karadimas
+Kapusniak
+Kapraun
+Kappe
+Kappa
+Kapitula
+Kapfer
+Kapelke
+Kapa
+Kaopua
+Kantarian
+Kanta
+Kanoza
+Kannard
+Kanish
+Kaniecki
+Kanevsky
+Kaner
+Kandra
+Kanda
+Kanatzar
+Kanable
+Kamph
+Kamnik
+Kammes
+Kammerdiener
+Kamerad
+Kamelamela
+Kamealoha
+Kame
+Kamb
+Kaluzny
+Kalupa
+Kaluna
+Kaltved
+Kalter
+Kalscheuer
+Kalmus
+Kalmer
+Kalland
+Kalima
+Kalichman
+Kalfa
+Kalbaugh
+Kakudji
+Kaitz
+Kainoa
+Kailey
+Kaiama
+Kahrer
+Kahola
+Kahana
+Kagay
+Kafel
+Kaetzel
+Kaesemeyer
+Kaer
+Kaea
+Kaduk
+Kadis
+Kaderlik
+Kade
+Kacik
+Kachikian
+Kacerski
+Kaboos
+Kabba
+Kaaz
+Kaauamo
+Juza
+Justino
+Justason
+Jurs
+Jurisch
+Jurgensmeier
+Jurden
+Jura
+Jungling
+Julye
+Juluke
+Julock
+Julias
+Julen
+Jufer
+Juedes
+Jubic
+Juariqui
+Juaire
+Jozsa
+Joulwan
+Jostes
+Josten
+Josich
+Josias
+Joshlin
+Josefy
+Josef
+Jorski
+Jorn
+Jorinscay
+Jorda
+Jons
+Jongeling
+Jongebloed
+Jondle
+Jolls
+Johnshoy
+Johnico
+Johanek
+Jirjis
+Jiran
+Jimmison
+Jill
+Jewels
+Jevtic
+Jetty
+Jesmer
+Jes
+Jerone
+Jerko
+Jenschke
+Jenquin
+Jennins
+Jennelle
+Jenison
+Jendrick
+Jeminez
+Jellis
+Jekot
+Jekel
+Jehl
+Jebb
+Jeavons
+Jeanneret
+Jeane
+Jeancharles
+Jeanbaptise
+Jaworowicz
+Javellana
+Jaurigui
+Jauch
+Jastrzebski
+Jass
+Jasmine
+Jarzembowski
+Jarver
+Jarosh
+Jaroscak
+Jarnesky
+Jares
+Jarell
+Jaradat
+Jarad
+Jaquins
+Janulewicz
+Jansing
+Janrhett
+Janowicz
+Janosek
+Jannetti
+Jannell
+Janeczko
+Jandron
+Janczunski
+Jancik
+Janacek
+Jamwant
+Jamili
+Jakovac
+Jagoe
+Jaffy
+Jaeschke
+Jaenke
+Jacque
+Jacobos
+Jackovitz
+Jackola
+Jackley
+Jacka
+Jacckson
+Jablonsky
+Jabiro
+Jabaay
+Jaap
+Iyengar
+Iwanowski
+Iwanejko
+Ivon
+Iverslie
+Ivanov
+Ivancich
+Iturralde
+Ittner
+Israelsen
+Israels
+Ismay
+Isleib
+Isita
+Isiordia
+Ising
+Isidore
+Isbill
+Isagawa
+Isacs
+Isaacsen
+Irzyk
+Irizzary
+Irineo
+Irimata
+Ireton
+Irestone
+Iozzo
+Iozzi
+Iopa
+Intrabartolo
+Intihar
+Insko
+Insana
+Inocente
+Ink
+Inhulsen
+Ingole
+Inches
+Inafuku
+Imperatore
+Imgrund
+Imbimbo
+Imbier
+Imaino
+Ilse
+Illuzzi
+Illian
+Ilic
+Ilasin
+Ilagan
+Iker
+Ihnat
+Ihm
+Igwe
+Igtanloc
+Ifversen
+Iese
+Ieng
+Ienco
+Idemoto
+Icard
+Iborra
+Ible
+Iberg
+Ibbetson
+Ibale
+Iavarone
+Iatarola
+Iacovino
+Iacopino
+Iacobellis
+Iachetta
+Hysom
+Hymowitz
+Hymon
+Hymen
+Hylands
+Hych
+Huy
+Huval
+Hutmacher
+Huszar
+Hustace
+Hussien
+Huskinson
+Husfelt
+Husenaj
+Husch
+Hurtig
+Hurtgen
+Huro
+Hurne
+Hurlston
+Hupman
+Huor
+Hunzelman
+Hunsperger
+Hunneyman
+Hunckler
+Humphrys
+Humphers
+Humetewa
+Humeniuk
+Humenik
+Hulstrand
+Hullings
+Hulitt
+Hulick
+Huland
+Huiting
+Hugron
+Hufstedler
+Huffner
+Huezo
+Huettman
+Huereca
+Huenink
+Huelse
+Hueckman
+Hudgeons
+Hudach
+Huckstadt
+Huckle
+Huckabey
+Hubschmitt
+Hubin
+Hubertus
+Hubby
+Hubbel
+Huban
+Huaman
+Hsun
+Hsiang
+Hrapski
+Hoznour
+Hoyman
+Howkins
+Howick
+Howatt
+Hovorka
+Hovick
+Hovanesian
+Hounchell
+Houf
+Hotton
+Hottes
+Hotrum
+Hotelling
+Hotaki
+Hostoffer
+Hosterman
+Hosteller
+Hospkins
+Hospelhorn
+Hoscheit
+Hoschander
+Horstead
+Horris
+Hornoff
+Hornberg
+Hornandez
+Hornack
+Hormell
+Horikoshi
+Horigan
+Horger
+Hoppins
+Hopperstad
+Hopko
+Hootsell
+Hoopingarner
+Hookano
+Hooghkirk
+Hoofard
+Hoock
+Honsinger
+Honour
+Honnette
+Honnerlaw
+Honma
+Honkanen
+Hongach
+Honeycott
+Hondorp
+Honchell
+Honas
+Honanie
+Homsher
+Homestead
+Holze
+Holtorf
+Holthus
+Holster
+Holsonback
+Holom
+Hollinrake
+Hollidge
+Hollerman
+Hollendonner
+Hollberg
+Holk
+Holian
+Holes
+Holecz
+Holec
+Holdvogt
+Hokutan
+Hok
+Hoiness
+Hoilman
+Hohiudden
+Hohensee
+Hohaia
+Hogelin
+Hogatt
+Hogarty
+Hoftiezer
+Hoffstatter
+Hoffnagle
+Hoffeditz
+Hoffart
+Hoerl
+Hoefel
+Hodos
+Hodnefield
+Hockins
+Hockenbrock
+Hocke
+Hochard
+Hocate
+Hobler
+Hober
+Hoben
+Hobell
+Hobden
+Hoagberg
+Hnyda
+Hlavka
+Hladik
+Hladek
+Hitchen
+Hislope
+Hirschberg
+Hirneise
+Hirn
+Hirliman
+Hirleman
+Hirao
+Hippenstiel
+Hintson
+Hint
+Hinley
+Hinh
+Hinebaugh
+Hindson
+Hinderberger
+Himmelmann
+Himanga
+Him
+Hilston
+Hilstad
+Hilser
+Hilsendager
+Hilsenbeck
+Hilscher
+Hilsabeck
+Hilpert
+Hilman
+Hillerud
+Hillebrano
+Hillebrandt
+Hilland
+Hilgers
+Hilgeman
+Hilfiker
+Hildago
+Hilda
+Hilbrand
+Hikel
+Highbaugh
+Higgons
+Higgenbottom
+Hiersche
+Hierholcer
+Hiedeman
+Hiday
+Hickethier
+Hichens
+Hibbitt
+Heyduck
+Hewko
+Hevron
+Heuwinkel
+Heuvelmann
+Heusner
+Heung
+Heuett
+Heuck
+Hettinga
+Hessey
+Hespen
+Hescock
+Heschke
+Hervig
+Hertzel
+Herston
+Herstad
+Hershkop
+Hershelman
+Herschelman
+Herriges
+Herres
+Herrarte
+Herpich
+Hernanez
+Hernanadez
+Hernan
+Hermenau
+Hermanowicz
+Herkstroeter
+Herkenratt
+Herera
+Herendeen
+Herauf
+Henstrom
+Hense
+Henrity
+Hennigh
+Hennies
+Henneberry
+Henkey
+Henjes
+Hengl
+Hengen
+Henfling
+Henerson
+Henein
+Hendrik
+Hendricksen
+Hendeson
+Henderso
+Henderlite
+Hemon
+Hemmann
+Hemker
+Hemesath
+Hemani
+Helweg
+Helverson
+Helseth
+Helquist
+Helom
+Helmstetter
+Helmsing
+Hellweg
+Hellmich
+Helgager
+Helgaas
+Helfenbein
+Helems
+Helem
+Helde
+Heiting
+Heither
+Heisdorffer
+Heiro
+Heirendt
+Heinzig
+Heiniger
+Heingartner
+Heimlicher
+Heimburger
+Heiken
+Heidtman
+Heidrich
+Heidi
+Heidelberger
+Heidebrecht
+Heick
+Heibult
+Heholt
+Heggood
+Heeth
+Heers
+Heern
+Heerkes
+Hedtke
+Hedspeth
+Hedon
+Hedinger
+Hecke
+Hechinger
+Hebeisen
+Heatherton
+Heartsill
+Heagney
+Heafey
+Headly
+Headland
+Headlam
+Headington
+Heade
+Hazy
+Hazim
+Haza
+Haynam
+Hayertz
+Haydt
+Haxby
+Hawse
+Hawkinberry
+Hawe
+Havlin
+Havir
+Havelka
+Hauxwell
+Hautan
+Hausrath
+Hauptmann
+Haughn
+Hauersperger
+Hatzenbihler
+Hattley
+Hatta
+Hatori
+Hathorne
+Hatchitt
+Hatchet
+Hatada
+Hastin
+Hastedt
+Hassing
+Hassenger
+Hassanein
+Hasker
+Haskel
+Hashaway
+Hasenfuss
+Hasenfratz
+Hascup
+Hasas
+Hartwigsen
+Hartrum
+Hartquist
+Hartory
+Hartlen
+Hartleben
+Hartinger
+Harsin
+Harritt
+Harriage
+Harpham
+Harnos
+Harnist
+Harleman
+Harlee
+Harke
+Hargers
+Hardter
+Hardsock
+Hardnette
+Hardine
+Hardi
+Hardges
+Harderman
+Harde
+Hardan
+Harcar
+Harbater
+Harapat
+Harang
+Haq
+Hanzl
+Hansome
+Hansman
+Hansis
+Hansing
+Hanoa
+Hanninen
+Hannaway
+Hannawalt
+Hanmer
+Hankison
+Hanible
+Hanenberger
+Haneke
+Hanebutt
+Handzlik
+Handsom
+Handkins
+Handke
+Handin
+Hanback
+Hanawalt
+Hanavan
+Hamsik
+Hamonds
+Hammette
+Hammerman
+Hammacher
+Hamlette
+Hamiltan
+Hamidi
+Hamff
+Hamett
+Hamersly
+Hamers
+Hamdn
+Hamden
+Hamberry
+Hamara
+Hamacher
+Halyk
+Haltiwanger
+Halstrom
+Halse
+Halpert
+Halnon
+Hallo
+Halliman
+Hallemeyer
+Hallack
+Halima
+Halick
+Haldi
+Halcott
+Halbershtam
+Halajian
+Halaas
+Hakey
+Haitz
+Hairell
+Haims
+Haifa
+Hahnert
+Haggin
+Haggerton
+Haggermaker
+Hagey
+Hafferkamp
+Haferkamp
+Haeuser
+Haessly
+Haese
+Haerter
+Haering
+Haeder
+Hadvab
+Hadsall
+Hadler
+Hadesty
+Haddenham
+Hadaller
+Hacopian
+Hackl
+Hackerott
+Hacken
+Hachting
+Haboush
+Hable
+Habig
+Habibi
+Haberstroh
+Habenicht
+Haaz
+Haakenstad
+Haage
+Gyllensten
+Gwilt
+Gwillim
+Guzon
+Guzewicz
+Guye
+Gutzler
+Guttormson
+Gutsche
+Gutjahr
+Gutgesell
+Gutenberg
+Gustitus
+Gussow
+Gusmar
+Gushi
+Gushard
+Gurwell
+Gurske
+Gurrero
+Gurin
+Gurecki
+Guoan
+Gunzelman
+Gunyon
+Guntharp
+Gunstream
+Gungor
+Gundelach
+Gunawan
+Gumprecht
+Gumaer
+Gulston
+Gulnac
+Gulizio
+Gulbrandsen
+Guitano
+Guimares
+Guillebeau
+Guillary
+Guillama
+Guilfoos
+Guiggey
+Guiga
+Guieb
+Guidrey
+Guiab
+Guffanti
+Guerrini
+Guerrazzi
+Guerera
+Guenthur
+Guell
+Guedjian
+Gudmundsson
+Gucker
+Gubin
+Gubala
+Guba
+Guasp
+Guarriello
+Guarno
+Guarini
+Guanche
+Guagenti
+Gstohl
+Grzesik
+Grzebien
+Gryszowka
+Grymes
+Gruz
+Grustas
+Gruse
+Gruntz
+Grunert
+Grune
+Grunberg
+Grumney
+Grumbling
+Gruman
+Grulkey
+Gruiger
+Gruening
+Gruenewald
+Gruby
+Gruben
+Grubel
+Grubba
+Grriffin
+Groys
+Growell
+Grothaus
+Grosskreutz
+Groskreutz
+Grosclaude
+Groot
+Gronstal
+Gronquist
+Gronlund
+Gronitz
+Gronberg
+Grona
+Gromoll
+Grohowski
+Grohman
+Groetsch
+Groder
+Grobmyer
+Groberg
+Grivno
+Grivetti
+Grippen
+Grine
+Grimme
+Grills
+Grigoreas
+Griglen
+Griffitt
+Griffan
+Grieshop
+Grieshaber
+Griep
+Grieff
+Griebling
+Griblin
+Grev
+Greubel
+Gressmire
+Gresco
+Grenway
+Grensky
+Grennay
+Grenko
+Grenet
+Gremo
+Gremmels
+Gregware
+Gregus
+Greggory
+Gregan
+Greep
+Greenweig
+Greensfelder
+Greenhalge
+Greengo
+Greenbacker
+Greem
+Greder
+Greczkowski
+Grebner
+Greber
+Greason
+Gream
+Gravat
+Grauman
+Grauel
+Grassle
+Grasmick
+Grapp
+Granzella
+Granto
+Gransberry
+Granquist
+Granneman
+Granieri
+Granes
+Grandon
+Grandner
+Granai
+Grammont
+Gramble
+Graleski
+Grainey
+Grain
+Graichen
+Grahovac
+Grageda
+Gragas
+Graffney
+Graffagnino
+Grafals
+Gradley
+Gradias
+Gradford
+Grabowsky
+Grabonski
+Grabler
+Grabhorn
+Graap
+Gozman
+Goyen
+Goyda
+Gowey
+Gowda
+Govostes
+Govia
+Gour
+Gouldman
+Gouldie
+Gougis
+Gotts
+Gottemoeller
+Gottdenger
+Gotta
+Gotshall
+Gosvener
+Gostlin
+Gossow
+Gosson
+Gossling
+Gosset
+Gosey
+Gorrindo
+Gormanous
+Gormally
+Gorius
+Gorena
+Gorell
+Gordley
+Gordey
+Gorbea
+Goonen
+Goodmon
+Gonzelas
+Gonzalis
+Gonyou
+Gonsiewski
+Gonsar
+Goney
+Gomoran
+Gomoll
+Gollop
+Gollob
+Gollier
+Golik
+Golida
+Golias
+Golian
+Golia
+Golec
+Goldthorpe
+Goldhorn
+Goldhirsh
+Goldfuss
+Goldfeld
+Golderer
+Goldenstein
+Goldenman
+Golde
+Golbin
+Golackson
+Goicoechea
+Goffigan
+Goerlich
+Goepfarth
+Goepel
+Goeing
+Goehringer
+Godboldt
+Gochett
+Gochal
+Gocek
+Goblirsch
+Gnoza
+Gnegy
+Gnabah
+Gmernicki
+Glyn
+Glueckert
+Glowacky
+Glovinsky
+Gloston
+Gloshen
+Glos
+Glogowski
+Gloeckler
+Glimpse
+Glidwell
+Glesener
+Gleitz
+Gleckler
+Glebocki
+Gleber
+Glazner
+Glazebrook
+Glaves
+Glavan
+Glasby
+Gladysiewski
+Gladle
+Gladhart
+Gjeltema
+Givant
+Gius
+Giulioli
+Gitt
+Girres
+Girbach
+Girand
+Gip
+Giottonini
+Giorno
+Gionta
+Giombetti
+Gioffre
+Gioe
+Ginzel
+Ginsel
+Ginocchio
+Ginnis
+Ginard
+Gimse
+Gilzow
+Gilton
+Gilstad
+Gilomen
+Gilner
+Gilly
+Gillming
+Gillion
+Gillich
+Gillice
+Gille
+Giliberto
+Gilhuly
+Gilgan
+Gildemeister
+Gilcris
+Gigger
+Giffith
+Giffee
+Giff
+Gietz
+Giesel
+Giera
+Gibeaut
+Gibala
+Giasson
+Giarusso
+Giarrano
+Giaquinta
+Giannavola
+Giandomenico
+Gianandrea
+Giallorenzo
+Giacherio
+Giachelli
+Giacchi
+Ghebremicael
+Gezalyan
+Getzschman
+Getzlaff
+Gettens
+Gettelman
+Gestether
+Gesing
+Gesamondo
+Gerz
+Gerwin
+Gerveler
+Gertsema
+Gerthung
+Gerten
+Gertel
+Gerteisen
+Gerstenberger
+Gershkovich
+Gerney
+Germy
+Germana
+Gerich
+Gerdiman
+Gerckens
+Gerbig
+Georghiou
+Geoly
+Gentleman
+Gentges
+Gentelia
+Gensel
+Geniesse
+Genia
+Generalao
+Gemmiti
+Geml
+Gelner
+Gellings
+Gellinger
+Gelino
+Gelhar
+Gelfond
+Gelerter
+Gelder
+Gelbart
+Geisinsky
+Gehrki
+Gehm
+Geen
+Gederman
+Gede
+Gearn
+Geant
+Gazzara
+Gazitano
+Gazdik
+Gayanilo
+Gawthorp
+Gavit
+Gaviglia
+Gavett
+Gavan
+Gavagan
+Gausman
+Gaukroger
+Gaufusi
+Gaudier
+Gaudett
+Gauci
+Gatzow
+Gatta
+Gatheright
+Gatesy
+Gatesman
+Gastelo
+Gaschke
+Garwin
+Garter
+Gartenmayer
+Gartenhaus
+Garsjo
+Garroutte
+Garrettson
+Garrean
+Garre
+Garnham
+Garnache
+Garmire
+Garmen
+Garlett
+Garkow
+Garito
+Garinger
+Gargan
+Garcon
+Gapp
+Gantzler
+Gantvoort
+Gansert
+Gansen
+Ganns
+Gannetti
+Ganin
+Ganigan
+Gamotan
+Gammond
+Gamer
+Gamello
+Gambrill
+Gambold
+Gambee
+Gambardella
+Galven
+Galvani
+Galuszka
+Galuppo
+Galmore
+Gallusser
+Gallodoro
+Gallington
+Galleta
+Gallegoz
+Gallaugher
+Gallargo
+Galkin
+Galipo
+Galinis
+Galimberti
+Galic
+Galbiso
+Galathe
+Galassini
+Galanti
+Galano
+Galagher
+Gajeski
+Gajardo
+Gaiters
+Gails
+Gailliard
+Gaffer
+Gafanha
+Gaer
+Gadewoltz
+Gaden
+Gackle
+Gabrial
+Gabrenas
+Gabossi
+Gables
+Gabl
+Gabhart
+Gabeline
+Gabbamonte
+Fyler
+Fykes
+Fusner
+Fusillo
+Fushimi
+Fus
+Furtak
+Furblur
+Fundora
+Funderberg
+Fumero
+Fuls
+Fulham
+Fulco
+Fujimura
+Fujikake
+Fugueroa
+Fuger
+Fugatt
+Fuerstenau
+Fuerbringer
+Frymoyer
+Frymier
+Frymark
+Frutiger
+Frushour
+Fruman
+Fruin
+Frugoli
+Fruehauf
+Froyd
+Frosto
+Frontis
+Frontiero
+Fronick
+Froneberger
+Frohberg
+Froebe
+Frobish
+Frittz
+Fritchley
+Fritchey
+Frisinger
+Frisell
+Frija
+Friehauf
+Friedenthal
+Friebel
+Freundlich
+Fret
+Frerich
+Frens
+Freker
+Freiseis
+Freimark
+Freilino
+Freiheit
+Freiermuth
+Freidin
+Freemantle
+Freeh
+Freedlander
+Freeders
+Freeburger
+Fredregill
+Frederique
+Freckleton
+Frecker
+Frazzano
+Frauenfelder
+Frattali
+Fratta
+Fratrick
+Fratercangelo
+Frasso
+Frashure
+Fraschilla
+Franzman
+Franzini
+Franza
+Franty
+Fransisco
+Franpton
+Frankson
+Frankland
+Frankiewicz
+Frankart
+Frangione
+Franchini
+Francescone
+Fralic
+Fraklin
+Frair
+Fragosa
+Fradkin
+Fracasso
+Foyer
+Foxhoven
+Fowlie
+Fowley
+Fowlar
+Fower
+Foute
+Foussell
+Fouquette
+Founds
+Fougner
+Fosmire
+Fosher
+Fosbrook
+Fortun
+Forss
+Forsmann
+Forslin
+Forsee
+Forpahl
+Fornili
+Fornier
+Fornaro
+Formichelli
+Formaggioni
+Forkum
+Forkell
+Foriest
+Forgrave
+Foresta
+Forejt
+Foreback
+Forcum
+Forcht
+Forchione
+Forch
+Forberg
+Forbach
+Fonua
+Fonteno
+Fonteneau
+Fongvongsa
+Fondriest
+Fondaw
+Fonck
+Fohl
+Foglio
+Foersterling
+Foddrell
+Focke
+Flugum
+Flucas
+Fluaitt
+Floss
+Florendo
+Floras
+Floer
+Flockhart
+Flockerzi
+Floan
+Flin
+Fliger
+Flieller
+Fleurilus
+Flenord
+Fleniken
+Flenaugh
+Flemmon
+Flemm
+Fleites
+Fleischner
+Fleckles
+Flechas
+Flauding
+Flatter
+Flato
+Flanner
+Flanegan
+Flammang
+Flakne
+Flaker
+Flagiello
+Fladung
+Flachs
+Flaa
+Fiwck
+Fitzrandolph
+Fitzherbert
+Fitzgerrel
+Fitsgerald
+Fisser
+Fishell
+Fischl
+Fischhaber
+Fischel
+Fiscella
+Fiscel
+Firpi
+Firenze
+Fiorilli
+Fiorica
+Finwall
+Finklestein
+Fingerson
+Fingerman
+Fineout
+Finello
+Finell
+Findlen
+Finco
+Filthaut
+Filpus
+Filo
+Filla
+Fili
+Fil
+Figiel
+Figgeurs
+Figert
+Fietek
+Fiest
+Fieser
+Fiesel
+Fickbohm
+Ficht
+Ficchi
+Fialho
+Fial
+Feyh
+Feyereisen
+Feuss
+Feusier
+Fette
+Festini
+Fest
+Fesko
+Fertik
+Ferrusi
+Ferrone
+Ferrio
+Ferringo
+Ferries
+Ferrie
+Ferrett
+Ferrato
+Ferrario
+Ferraraccio
+Ferranto
+Ferr
+Ferouz
+Fernette
+Fernanders
+Ferkel
+Feret
+Ferer
+Ferenz
+Fenrich
+Fenniman
+Fennig
+Fenison
+Fendrick
+Fendlason
+Fend
+Fenbert
+Felver
+Feltham
+Felonia
+Felling
+Fellezs
+Felizardo
+Felio
+Felicien
+Felicia
+Felicano
+Feliberty
+Feistner
+Feister
+Feintuch
+Feilds
+Feighner
+Feierman
+Fehrs
+Fegueroa
+Fegles
+Fegette
+Feerick
+Feela
+Feehly
+Feehery
+Fedorko
+Fedie
+Fedezko
+Fedewa
+Federkeil
+Fecto
+Fechtig
+Fecher
+Featheroff
+Feagans
+Fazzari
+Faycurry
+Fawson
+Fawler
+Favuzzi
+Favro
+Favian
+Favazza
+Fausey
+Faus
+Faupel
+Fattore
+Fatora
+Fathy
+Fathree
+Fatheree
+Fassinger
+Faske
+Farug
+Fars
+Farnese
+Farkus
+Farinha
+Faren
+Faraimo
+Farahkhan
+Faragher
+Fanti
+Fanter
+Fantazia
+Fantauzzo
+Fansher
+Fandino
+Fanatia
+Famageltto
+Falzon
+Fallow
+Fallenstein
+Falencki
+Falcioni
+Falci
+Failey
+Failde
+Faigley
+Faidley
+Fahrni
+Fahrlander
+Fahrenthold
+Fahning
+Fago
+Fagle
+Fagerquist
+Fagerlund
+Fageraes
+Facello
+Ezzelle
+Eyton
+Eyestone
+Exton
+Exantus
+Evjen
+Evilsizor
+Evertt
+Evertsen
+Eversmeyer
+Everroad
+Everline
+Everet
+Evartt
+Evansky
+Evancho
+Eull
+Ettman
+Ettienne
+Ettel
+Etringer
+Eth
+Estronza
+Estrem
+Estrade
+Estok
+Estle
+Estimable
+Estess
+Estella
+Estanislau
+Essix
+Essency
+Esquinaldo
+Espiridion
+Espinel
+Esperon
+Espenlaub
+Espejel
+Esparsen
+Esmont
+Esmon
+Esmay
+Esmaili
+Eskins
+Eskind
+Eshmon
+Esfahani
+Escober
+Escanlar
+Erz
+Ersery
+Eros
+Ernster
+Erlebach
+Eriks
+Erichson
+Erger
+Eredia
+Erdos
+Ercole
+Ercolano
+Erazmus
+Eraso
+Epel
+Eovaldi
+Ensz
+Ensel
+Enock
+Ennes
+Enis
+Engnath
+Engfer
+Engelmeyer
+Engelberg
+Engard
+Endris
+Endreson
+Endorf
+Endersbe
+Ende
+Encino
+Emshwiller
+Empasis
+Emore
+Emmond
+Emiliano
+Emerling
+Emenaha
+Emde
+Emberling
+Emano
+Elway
+Elvey
+Eltringham
+Elter
+Elsken
+Elsheimer
+Elsaesser
+Elrick
+Elreda
+Elpert
+Elnicki
+Elmes
+Ellsmore
+Ellrod
+Ello
+Ellinghuysen
+Ellingham
+Ellingburg
+Elles
+Ellenbogen
+Elleby
+Ellcessor
+Ellamar
+Elke
+Elijah
+Eligio
+Elieff
+Elicker
+Elian
+Eliades
+Elhadi
+Elfenbein
+Elenbaas
+Eldringhoff
+Eld
+Elbie
+Eke
+Ekas
+Eisnaugle
+Eisiminger
+Eisenhaver
+Eisenhardt
+Eisenberger
+Eiselein
+Einwalter
+Eighmey
+Eidemiller
+Eickmeyer
+Eichstedt
+Eichenberg
+Eichberg
+Eibel
+Ehrisman
+Ehrenzeller
+Ehman
+Ehli
+Ehl
+Eheler
+Egwuohua
+Eglin
+Egler
+Egersdorf
+Egelston
+Efthimiou
+Eelkema
+Edu
+Edridge
+Edland
+Edenholm
+Edem
+Economou
+Eckmann
+Eckblad
+Eckardt
+Echternach
+Echter
+Ebrahimi
+Eberst
+Ebershoff
+Eberheart
+Ebbett
+Eayrs
+Eavey
+Eatough
+Eastling
+Eastern
+Easterlin
+Earthly
+Earing
+Eakles
+Eagleman
+Eacho
+Eaby
+Dzwonkowski
+Dzurnak
+Dzurilla
+Dziuba
+Dzinski
+Dziewanowski
+Dziekan
+Dyrstad
+Dydo
+Dvorsky
+Duyer
+Duttinger
+Dutchess
+Duston
+Dush
+Durward
+Dursteler
+Durpee
+Durough
+Durniok
+Durnan
+Durisseau
+Duris
+Duriga
+Durda
+Durboraw
+Dura
+Duquaine
+Duplessy
+Duplanti
+Dupes
+Duperre
+Dupaski
+Duos
+Dunshie
+Dunphe
+Dunnell
+Dunkinson
+Dunkerley
+Dunkan
+Dunemann
+Dunderman
+Duncans
+Dunahoe
+Dumouchel
+Dummett
+Dumeny
+Dumbar
+Dumar
+Dulan
+Dukett
+Duk
+Duis
+Duguette
+Dugre
+Dufrain
+Dufauchard
+Duesterhaus
+Duesterback
+Duerst
+Duenwald
+Dudzik
+Dudycha
+Dudenbostel
+Dudden
+Ducklow
+Duckey
+Duchnowski
+Duchane
+Duceman
+Dubovsky
+Dubler
+Duber
+Dubel
+Dubbert
+Drutman
+Drummey
+Drumbore
+Droy
+Drow
+Droubay
+Drorbaugh
+Dropinski
+Dronko
+Dronick
+Droggitis
+Drissel
+Driscol
+Drinen
+Driessen
+Driedric
+Dreuitt
+Drenning
+Drelick
+Drejka
+Dreiss
+Drebes
+Dratch
+Drakulic
+Drakos
+Draime
+Dragovich
+Dragich
+Draggett
+Dragg
+Drabicki
+Doyscher
+Doxbeck
+Downy
+Downhour
+Dowland
+Dowker
+Dowds
+Dowda
+Douyette
+Douthett
+Doughman
+Dougharty
+Douga
+Doudna
+Dotolo
+Dossman
+Dosh
+Dorsinville
+Dorsay
+Dorrill
+Dorosh
+Dornbrook
+Dorlando
+Dorio
+Dorie
+Dorcas
+Doporto
+Dopita
+Doorley
+Dooner
+Donton
+Dono
+Donnerberg
+Donnalley
+Donlyuk
+Donkle
+Donilon
+Doniger
+Donigan
+Doniel
+Doncaster
+Donatich
+Donaher
+Donah
+Donaghue
+Donaby
+Domowicz
+Domitrovich
+Dominowski
+Dominiak
+Domenice
+Dombek
+Domagalski
+Domagall
+Dolsen
+Dolmajian
+Dolley
+Dolinski
+Dolhun
+Dolfi
+Dolecek
+Dokovic
+Dok
+Dohrn
+Doerksen
+Doelger
+Doeberling
+Dody
+Dodimead
+Dodgion
+Dockum
+Dockerty
+Dochterman
+Dobrzykowski
+Dobrynski
+Dobrushin
+Dobrosky
+Dobrinin
+Dobison
+Dobbyn
+Dobbe
+Dlugos
+Ditucci
+Dittus
+Dittmann
+Dito
+Ditmars
+Disotell
+Disorda
+Disharoon
+Dischner
+Discala
+Disalvi
+Dirth
+Dirr
+Dirienzo
+Dipolito
+Dipilato
+Dipietrantoni
+Dipanfilo
+Dioneff
+Diomede
+Dinuzzo
+Dintino
+Dinsmoor
+Dinsdale
+Dinos
+Dinora
+Dinnendahl
+Dinkle
+Dininger
+Dingillo
+Dingie
+Dingell
+Dimitry
+Dimicco
+Dimezza
+Dimarzio
+Dimario
+Dimariano
+Dimanche
+Dilucca
+Dillis
+Dilliner
+Dillin
+Dillashaw
+Dilillo
+Dilg
+Dilella
+Diker
+Digiouanni
+Digeorgio
+Difronzo
+Difrancisco
+Dietterick
+Diestler
+Dies
+Dierkes
+Diekema
+Diederichs
+Dieball
+Didway
+Didonatis
+Didomizio
+Didio
+Didato
+Dicosmo
+Dicorpo
+Dicocco
+Diclaudio
+Dichiaro
+Dible
+Diblase
+Dibiasi
+Dibbern
+Diano
+Diani
+Diangelis
+Diamantopoulo
+Diaco
+Dhruva
+Dheel
+Dharas
+Dezalia
+Deyak
+Deya
+Dewolff
+Dewick
+Dewese
+Dewater
+Devot
+Devost
+Devis
+Devilliers
+Devery
+Deveny
+Devenny
+Develice
+Devasier
+Devarona
+Devanski
+Devai
+Deus
+Dettorre
+Dettor
+Detrolio
+Detrich
+Detillion
+Deteso
+Determann
+Deterline
+Deterding
+Detchon
+Detaeye
+Destina
+Destefani
+Desruisseaux
+Desormeau
+Desonia
+Desmore
+Desko
+Desimas
+Desher
+Deshayes
+Deschene
+Desantos
+Desando
+Desamparo
+Desalvatore
+Derx
+Deruiter
+Derosie
+Derogatis
+Derman
+Derkas
+Derivan
+Derington
+Derienzo
+Derian
+Dereus
+Derenzi
+Derentis
+Derderian
+Derastel
+Deraps
+Dequinzio
+Deprato
+Depont
+Depiro
+Depierro
+Depeyster
+Deonarine
+Deocampo
+Denzine
+Denwood
+Denos
+Denooyer
+Denomme
+Denoia
+Dennig
+Denjen
+Denisco
+Denick
+Denholm
+Denfip
+Deneui
+Denetclaw
+Denet
+Denery
+Demuzio
+Demske
+Dempewolf
+Demorrett
+Demorizi
+Demny
+Demiter
+Demilt
+Demik
+Demien
+Demianczyk
+Demetrakos
+Demer
+Dembek
+Demauro
+Demase
+Demart
+Demarino
+Deluzio
+Delullo
+Delucian
+Deltufo
+Deltora
+Delsoin
+Delsavio
+Delross
+Delperdang
+Delpaggio
+Delosier
+Delonge
+Delonais
+Deloge
+Delmendo
+Dellwo
+Dellum
+Dellosso
+Delliveneri
+Dellefave
+Dellarose
+Dellapenta
+Dellamonica
+Delgoda
+Delekta
+Delegado
+Deldonno
+Delco
+Delce
+Delbene
+Delavergne
+Delashmutt
+Delapuente
+Delaporte
+Delana
+Delallo
+Delahay
+Delagol
+Delagado
+Delabarre
+Dekruif
+Dekoning
+Dekeyzer
+Dejoseph
+Dejardin
+Dejarden
+Deister
+Deigado
+Deichmann
+Deichman
+Dehm
+Dehlinger
+Dehl
+Dehetre
+Dehaney
+Dehaas
+Degrood
+Degrass
+Degrande
+Degooyer
+Degnim
+Deglandon
+Degenfelder
+Degenaro
+Degear
+Degagne
+Defrang
+Defrain
+Defosset
+Defosse
+Defont
+Defir
+Defayette
+Deerdoff
+Deely
+Dedrickson
+Dednam
+Dederich
+Decurtis
+Decourt
+Decourcey
+Decock
+Declerk
+Decius
+Dechavez
+Dech
+December
+Decarvalho
+Decarmine
+Decaire
+Decaen
+Debrosse
+Debreto
+Debrecht
+Debrae
+Debore
+Debien
+Debenedictis
+Debarge
+Debardelaben
+Debaets
+Deasis
+Dears
+Dearruda
+Dearring
+Dearinger
+Dearin
+Dearcos
+Deanes
+Deakyne
+Dazzi
+Dazi
+Dayao
+Dawkin
+Davolt
+Davise
+Davine
+Davidsmeyer
+Davidowicz
+Davaz
+Davari
+Davance
+Dauster
+Dause
+Daulerio
+Daughters
+Daugereau
+Daubney
+Datamphay
+Dasouza
+Daskal
+Dashno
+Dashne
+Dasen
+Daschofsky
+Dasch
+Darwich
+Darvish
+Darveau
+Darting
+Darthard
+Darron
+Daron
+Darnstaedt
+Darmody
+Darmiento
+Darington
+Dariano
+Daria
+Dardenne
+Darakjian
+Danyow
+Dannis
+Danniels
+Danni
+Dannelly
+Dannelley
+Dannatt
+Daniely
+Dangelis
+Danese
+Daner
+Dandoy
+Danco
+Danca
+Danas
+Damrell
+Damone
+Damms
+Damme
+Dalporto
+Daloisio
+Dalmata
+Dallison
+Dallam
+Dallago
+Dalegowski
+Dalecki
+Daku
+Daking
+Daken
+Dajer
+Dajani
+Daidone
+Dahlka
+Dagres
+Dago
+Dager
+Dafonte
+Dada
+Daczewitz
+Dach
+Czysz
+Czubakowski
+Czartoryski
+Czapiewski
+Cyrnek
+Cyree
+Cygrymus
+Cwikla
+Cwalinski
+Cutrera
+Cuther
+Cutchember
+Cushner
+Cusenza
+Curreri
+Curlis
+Curio
+Curimao
+Curia
+Curey
+Cunio
+Cumoletti
+Cumberlander
+Culpit
+Culloton
+Cuffy
+Cuffman
+Cuddington
+Cucuta
+Cucufate
+Cubine
+Cubano
+Cuadras
+Csuhta
+Crutison
+Cruther
+Crusinberry
+Crummell
+Crumly
+Cruff
+Crozat
+Crossmon
+Crosiar
+Crookshank
+Crookes
+Cronoble
+Croner
+Cromeans
+Crolley
+Crofutt
+Crockette
+Crivelli
+Crivaro
+Cristino
+Criste
+Crissey
+Crisalli
+Criley
+Cribari
+Crewe
+Creselious
+Crescenti
+Crepps
+Crenwelge
+Creitz
+Cregin
+Cregger
+Creekbaum
+Credi
+Crebs
+Crayford
+Cravy
+Cravalho
+Crauswell
+Crathers
+Crask
+Crapp
+Crape
+Crapanzano
+Cranson
+Crans
+Crannell
+Crandal
+Craigwell
+Craigmyle
+Crafter
+Cradler
+Coxwell
+Coxen
+Cowlin
+Covitz
+Coventon
+Coutre
+Coutinho
+Coutermarsh
+Courton
+Courseault
+Courrege
+Courey
+Coulon
+Coulibaly
+Couden
+Coton
+Coste
+Cossett
+Cosman
+Cosma
+Coslow
+Cosico
+Coshow
+Corwell
+Corvo
+Corujo
+Cortopassi
+Cortinez
+Cortijo
+Corrio
+Corrington
+Corriher
+Corridan
+Corrga
+Correla
+Corping
+Corpe
+Coroniti
+Cornn
+Cornmesser
+Cornella
+Corneille
+Corkron
+Corf
+Coreen
+Cordiero
+Cordew
+Cordenas
+Corcuera
+Corbley
+Coray
+Coraham
+Copstead
+Copsey
+Copping
+Coppes
+Copney
+Coopper
+Cooperider
+Coopage
+Coonse
+Cookerly
+Conwright
+Contreraz
+Continenza
+Contes
+Consuelo
+Constine
+Constanzo
+Constantin
+Constancio
+Consentino
+Conradt
+Conour
+Conoley
+Conney
+Connerat
+Conlogue
+Conforme
+Confalone
+Coneway
+Condroski
+Condina
+Condiff
+Condi
+Conchado
+Conch
+Concatelli
+Conaughty
+Commerford
+Comissiong
+Cominski
+Cominotti
+Comar
+Colschen
+Colpi
+Colpa
+Colony
+Collons
+Collon
+Collicott
+Collea
+Collari
+Colker
+Colier
+Colesar
+Colemen
+Colecchi
+Colcher
+Colchado
+Coklow
+Cokel
+Cohick
+Cofone
+Coffinberger
+Coffell
+Coffel
+Codispot
+Codilla
+Cocroft
+Cockerhan
+Cochren
+Cochenour
+Cobetto
+Cobar
+Coalter
+Clyman
+Cluver
+Clusky
+Clunes
+Clukies
+Clowerd
+Clouatre
+Clossin
+Cloos
+Clokey
+Clinkinbeard
+Cliffton
+Clibon
+Clevland
+Cleverley
+Clesca
+Clerc
+Clemenza
+Cleath
+Cleasby
+Cleal
+Clavijo
+Clater
+Claros
+Claghorn
+Clacher
+Clabo
+Civil
+Cittadini
+Citroni
+Cissel
+Cisar
+Cirella
+Circelli
+Ciprian
+Cipcic
+Ciotta
+Cinnamond
+Cinkan
+Cinco
+Cinar
+Cimorelli
+Ciminera
+Cilenti
+Cihak
+Cieloszyk
+Cidre
+Cicen
+Cicali
+Cibik
+Ciavardini
+Cianfrani
+Cianciola
+Ciallella
+Ciaffone
+Chyle
+Chy
+Churchfield
+Churape
+Chuma
+Chulla
+Chueng
+Chubicks
+Chrystal
+Chrosniak
+Chriswell
+Christopoulos
+Christi
+Christerson
+Christenbury
+Chowenhill
+Chowansky
+Choudhary
+Chor
+Chopton
+Cholula
+Chollett
+Choinski
+Chocron
+Chockley
+Chochrek
+Choates
+Chlebus
+Chiz
+Chitrik
+Chisman
+Chiphe
+Chiola
+Chiodi
+Chinault
+Chime
+Chimal
+Chilsom
+Chillo
+Chicles
+Chicharello
+Chicalace
+Chiariello
+Chiappari
+Chhan
+Chham
+Chez
+Chevis
+Cheverton
+Cheverez
+Cheu
+Chessman
+Cherubini
+Cherrin
+Cheroki
+Cherny
+Chernich
+Chernesky
+Cheranichit
+Cheeseboro
+Chech
+Cheam
+Chavoustie
+Chavies
+Chaumont
+Chaulklin
+Chatampaya
+Chasson
+Chassaniol
+Chary
+Charvet
+Charry
+Chari
+Chararria
+Chappo
+Chappa
+Chapmond
+Chaplik
+Chapen
+Chanthasene
+Chanler
+Chanco
+Chamul
+Champaco
+Chalupa
+Challinor
+Challa
+Chalender
+Chaknis
+Chakkalakal
+Chaisty
+Chaddick
+Chaboya
+Chaberek
+Chabbez
+Cevera
+Cerverizzo
+Cerventez
+Cervantsz
+Cerva
+Cerroni
+Cerri
+Cerrello
+Cerone
+Cernuto
+Cernota
+Cerminaro
+Cerf
+Ceretti
+Cerceo
+Cerasuolo
+Ceraso
+Cerasi
+Cerar
+Ceraos
+Cepin
+Cepas
+Centi
+Cendana
+Cendan
+Cellar
+Celeya
+Ceder
+Cecot
+Cazel
+Cazaree
+Cawon
+Cawein
+Cavrak
+Caveness
+Cavalaris
+Cavaiani
+Cauterucci
+Caughorn
+Caughell
+Cauazos
+Catts
+Cattanach
+Catrini
+Catozzi
+Catignani
+Catholic
+Catherson
+Catherine
+Cathell
+Catello
+Catchpole
+Catanzano
+Casuscelli
+Castros
+Castrey
+Castongvay
+Castillion
+Castelum
+Castells
+Castellion
+Cassler
+Cassino
+Cassilano
+Cassiano
+Cassetty
+Cassens
+Cassells
+Cassavaugh
+Cassagne
+Cassa
+Casolary
+Casmore
+Casley
+Caska
+Casis
+Casini
+Cashour
+Cashmer
+Cashett
+Casement
+Casciato
+Casavez
+Casasola
+Casarz
+Casar
+Casana
+Casales
+Carvill
+Carvallo
+Cartner
+Carrousal
+Carrizo
+Carretta
+Carrethers
+Carrao
+Carran
+Carpen
+Caroselli
+Carolla
+Carnillo
+Carnegia
+Carmin
+Carmickel
+Carlini
+Carland
+Carknard
+Carioscia
+Carina
+Carideo
+Carfrey
+Cardinalli
+Cardiff
+Cardazone
+Carbonella
+Carbery
+Carbee
+Caravetta
+Caravati
+Caramelo
+Caramella
+Caraig
+Carabine
+Cara
+Capristo
+Capri
+Cappellini
+Caporiccio
+Capicotto
+Capestro
+Capener
+Capek
+Capas
+Capaccino
+Caoagdan
+Canwell
+Cantella
+Cantakis
+Canson
+Cansino
+Cansibog
+Cannistraro
+Canner
+Caneza
+Caney
+Caneva
+Canetta
+Canestraro
+Candozo
+Candlish
+Candell
+Canant
+Canalez
+Can
+Camus
+Campora
+Campobasso
+Campble
+Campau
+Campain
+Camlin
+Camisa
+Camerino
+Camerano
+Camenisch
+Camelin
+Cameli
+Cambia
+Camareno
+Camancho
+Camack
+Calvan
+Calumag
+Caltagirone
+Calowell
+Callnan
+Callington
+Calliham
+Calligaro
+Caller
+Callar
+Callam
+Callagy
+Callagher
+Callado
+Caliman
+Caldron
+Caldoron
+Caldarera
+Calcao
+Calaf
+Cakmak
+Cajulus
+Cajka
+Caivano
+Caires
+Caire
+Caiozzo
+Cains
+Cainne
+Caimi
+Cagnon
+Cagno
+Cagan
+Caffentzis
+Cafasso
+Caez
+Caddigan
+Caddel
+Cacatian
+Cabugos
+Cabon
+Cabarcas
+Cabanillas
+Cabanela
+Cabam
+Bywaters
+Bystron
+Byse
+Byous
+Bynun
+Byczek
+Bybel
+Byal
+Buzza
+Buzo
+Buzis
+Buvinghausen
+Butzke
+Buttross
+Buttray
+Buttke
+Buttitta
+Butenhoff
+Busscher
+Busk
+Busitzky
+Bushweller
+Bushrod
+Bushfield
+Buschur
+Busacca
+Burzlaff
+Burvine
+Burtts
+Burtschi
+Burtell
+Bursik
+Burrs
+Burras
+Burows
+Burnie
+Burnash
+Burmside
+Burm
+Burly
+Burlson
+Burlile
+Burlaza
+Burlage
+Burkstrand
+Burkly
+Burklow
+Burkin
+Burian
+Burgs
+Burgoa
+Burgey
+Burgees
+Burfeind
+Burdzel
+Burchinal
+Burbine
+Buratti
+Buonassisi
+Buonaiuto
+Buntz
+Bunts
+Buntenbach
+Bunson
+Bunda
+Bumpaus
+Bumbalo
+Bumbaca
+Bullivant
+Bullin
+Bulisco
+Bulik
+Buley
+Bulat
+Bukowiecki
+Builes
+Buhrke
+Buhlig
+Bugh
+Buffone
+Buenviaje
+Bueler
+Buehlman
+Budzik
+Budy
+Budrovich
+Budish
+Budiao
+Budhu
+Buden
+Buddy
+Bud
+Buczko
+Bucknor
+Buckmeon
+Buckless
+Buckett
+Buckaloo
+Buchwalter
+Buchmiller
+Buchmeier
+Buchite
+Buchinsky
+Bucheli
+Buchann
+Buchal
+Bucaro
+Bubolz
+Buboltz
+Bubert
+Brzezicki
+Brzenk
+Brys
+Bryngelson
+Bryla
+Bryington
+Bruzewski
+Bruzek
+Brustmann
+Brusser
+Bruscato
+Brunzel
+Brunkhardt
+Brunick
+Brunetta
+Brunecz
+Bruna
+Brumaghim
+Bruker
+Bruin
+Brugliera
+Bruffee
+Brueske
+Bruegger
+Bruechert
+Bruckmeier
+Brroks
+Brozeski
+Broyle
+Brownlie
+Browman
+Broudy
+Brothen
+Broski
+Brosi
+Brookskennedy
+Brookie
+Bronston
+Broncheau
+Brommer
+Brola
+Broitzman
+Brohn
+Broglio
+Brogley
+Broers
+Broering
+Brodtmann
+Brodis
+Brodine
+Brodfuehrer
+Brodess
+Brodes
+Brockus
+Brockenberry
+Brociner
+Brochet
+Broadnay
+Brizeno
+Britts
+Brinley
+Brinkhaus
+Brinius
+Brininger
+Bringer
+Brindza
+Brindger
+Brinar
+Brilowski
+Brigner
+Brightharp
+Brighter
+Brienza
+Brienen
+Bridenbecker
+Brickson
+Breznay
+Brezinka
+Breyers
+Brevell
+Brettmann
+Bretos
+Bresser
+Brentz
+Brennick
+Brening
+Brendeland
+Brem
+Breiter
+Breihan
+Breidigan
+Bredlow
+Bredin
+Breckley
+Breckenstein
+Brebes
+Breaz
+Breaud
+Breath
+Bready
+Brazie
+Braunwarth
+Braunberger
+Brauman
+Braucks
+Brath
+Brasure
+Brasswell
+Brasseux
+Braskett
+Brasby
+Brantingham
+Bransfield
+Branseum
+Brano
+Brangers
+Brang
+Branes
+Brandstrom
+Brandorff
+Brandom
+Brandenburger
+Branck
+Brancaccio
+Bramuchi
+Bramlitt
+Bramel
+Bramasco
+Bram
+Brakke
+Brak
+Braget
+Bragado
+Brafman
+Bradmon
+Bradick
+Bradey
+Bradd
+Bracklin
+Brackbill
+Brabazon
+Braband
+Bozych
+Bozic
+Boyl
+Boyens
+Boyde
+Boyas
+Bowlick
+Bowle
+Bowcock
+Bouy
+Bouvia
+Bousum
+Bourraine
+Bourgon
+Bourbois
+Bouquin
+Boumthavee
+Boulger
+Boulch
+Boulais
+Boughn
+Bouges
+Boudle
+Boudjouk
+Boucouvalas
+Boucaud
+Bottrell
+Bottoni
+Bottella
+Bothner
+Botellio
+Boswink
+Bostow
+Bostain
+Bosson
+Bossier
+Bossey
+Bosold
+Boslet
+Boshnack
+Boshell
+Bosheers
+Bosefski
+Borza
+Boryszewski
+Borysewicz
+Borson
+Borseth
+Borroto
+Borrigo
+Borriello
+Borrello
+Borowicz
+Borovetz
+Borovec
+Borgelt
+Bordinger
+Bordas
+Bord
+Borcuk
+Borcher
+Borbridge
+Boothman
+Bookhardt
+Boocock
+Bonwell
+Bonsal
+Bonnoitt
+Bonnifield
+Bonnick
+Bonnel
+Bonker
+Bonita
+Boning
+Bonifield
+Boniface
+Bongle
+Bongivengo
+Bongio
+Bonge
+Bonett
+Bonebright
+Bondroff
+Bondoc
+Bonda
+Boncella
+Bonaventure
+Bonalumi
+Bonadona
+Bonaccorso
+Bonaccorsi
+Bompiani
+Bommer
+Bolvin
+Boluda
+Bolorin
+Bolon
+Bollom
+Bollettino
+Bolk
+Boliver
+Boline
+Bolieu
+Boliek
+Boleyn
+Boldul
+Boldery
+Bolante
+Bokor
+Boklund
+Bojanowski
+Boisuert
+Boislard
+Bohren
+Bohmann
+Bohlinger
+Bohart
+Boham
+Bogust
+Bogh
+Bogatay
+Bogany
+Boeving
+Boeshore
+Boesenberg
+Boerstler
+Boers
+Boenig
+Boelsche
+Boelke
+Boekhout
+Boekelman
+Boehner
+Boeckmann
+Bodwin
+Bodrey
+Bodman
+Bodiroga
+Bodford
+Bodensteiner
+Bodenheimer
+Boddorf
+Boddeker
+Bockskopf
+Bocchi
+Bocage
+Bobola
+Bobko
+Boben
+Boardway
+Boards
+Blyzes
+Blumenkranz
+Bloomgren
+Blong
+Blondeau
+Blommel
+Blois
+Bloem
+Blocklinger
+Blisset
+Blimka
+Bliler
+Bliese
+Blice
+Bleyer
+Blette
+Blesh
+Blender
+Blemel
+Bleifus
+Blechinger
+Bleattler
+Blazosky
+Blatti
+Blatteau
+Blatnik
+Blatchford
+Blankship
+Blankschan
+Blandy
+Blandino
+Blakeway
+Blakeborough
+Blaho
+Blackstar
+Blackgoat
+Blachly
+Blacher
+Blach
+Bizcassa
+Bizarro
+Bivings
+Bitsuie
+Bitsui
+Bitsko
+Bistodeau
+Bister
+Bisonette
+Bishel
+Bisconer
+Biscocho
+Biscahall
+Bisby
+Bisagna
+Birts
+Birnell
+Birkline
+Birkenhead
+Birenbaum
+Birckett
+Birckbichler
+Birchwood
+Biorkman
+Bimler
+Bilous
+Billinghurst
+Billey
+Billeter
+Billegas
+Billard
+Bilkiss
+Bile
+Bilcik
+Bigos
+Bignall
+Bigio
+Biggio
+Bigas
+Biffer
+Biffar
+Biesinger
+Bieschke
+Bierbrauer
+Bienfang
+Biehn
+Biederwolf
+Bieberle
+Biebel
+Bidon
+Bidner
+Bidgood
+Bidez
+Biderman
+Bickleman
+Bicklein
+Bicket
+Bicker
+Bickart
+Bichel
+Biard
+Bialik
+Bialczyk
+Bezner
+Beyrer
+Beylotte
+Beyerl
+Bevly
+Beulah
+Beul
+Betzel
+Betterman
+Betsinger
+Betschman
+Betita
+Bethurum
+Bethoney
+Beth
+Beston
+Besso
+Bessick
+Besio
+Beshear
+Besarra
+Bervig
+Bertus
+Bertrano
+Bertovich
+Bertolasio
+Bertog
+Bertinetti
+Bertelle
+Bertel
+Bertch
+Bertagnoli
+Berschauer
+Bersamin
+Bers
+Berri
+Berretti
+Berretta
+Berret
+Bernucho
+Bernt
+Bernstrom
+Berno
+Bernick
+Bernice
+Bernhagen
+Bernardoni
+Bernabo
+Bermers
+Berlove
+Berlinghof
+Berkhalter
+Berisha
+Bergseng
+Bergreen
+Bergholz
+Bergert
+Berez
+Beresnyak
+Berdes
+Beras
+Benzschawel
+Benzi
+Benya
+Benwell
+Benty
+Bentrup
+Bentele
+Benser
+Bennison
+Bennink
+Bennerson
+Bennerman
+Benitone
+Beniquez
+Benik
+Bengelsdorf
+Benell
+Beneduce
+Benecke
+Benear
+Bendzans
+Bendy
+Bendt
+Bendorf
+Bendolph
+Bendlage
+Benders
+Bendavid
+Benck
+Benassi
+Benari
+Benage
+Benadom
+Benabides
+Bembury
+Bemboom
+Bemberry
+Belyoussian
+Belveal
+Belsey
+Belongie
+Belone
+Belon
+Beloff
+Belluomini
+Belloma
+Bellmay
+Bellish
+Bellisario
+Bellingham
+Bellflower
+Bellfleur
+Bellerdine
+Bellemy
+Bellazer
+Belkowski
+Belich
+Belfiglio
+Beley
+Beldin
+Belback
+Belarde
+Belangia
+Bel
+Bekerman
+Beker
+Bek
+Beiswanger
+Beirise
+Behun
+Behning
+Behmer
+Behlen
+Begor
+Begg
+Beetley
+Bees
+Beermudez
+Beerling
+Beeck
+Bedsaul
+Bedoka
+Bednorz
+Becklund
+Beckerdite
+Beckendorf
+Beckenbach
+Bechthold
+Bechman
+Becherer
+Beavin
+Beauprez
+Beaumier
+Beauliev
+Beaugard
+Beaufait
+Beaudrie
+Beathe
+Beasmore
+Bearup
+Bearfield
+Beahn
+Beadnell
+Beadell
+Bazzel
+Bazzanella
+Bazelais
+Bazata
+Bazarte
+Baza
+Bayle
+Bayete
+Bawa
+Bavzee
+Bavard
+Bausley
+Baunleuang
+Baumgard
+Baumbusch
+Bauknight
+Baugham
+Bauers
+Bauermeister
+Baublitz
+Battistini
+Battiato
+Battiata
+Batters
+Battaglini
+Bathurst
+Bathrick
+Batel
+Batalona
+Basua
+Bastura
+Bastress
+Bastilla
+Bastidos
+Bastic
+Basten
+Bastedo
+Bastain
+Bassil
+Basset
+Bashinelli
+Basbas
+Baruth
+Barufaldi
+Bartylla
+Barts
+Bartrop
+Bartosz
+Bartosiak
+Bartolotto
+Bartolet
+Bartoldus
+Bartnett
+Bartlone
+Barthen
+Barthelman
+Bartenfield
+Bartczak
+Barsotti
+Barrocas
+Barrile
+Barrieau
+Barrer
+Barreira
+Barranger
+Barranca
+Barquera
+Barnscater
+Barnfield
+Barncastle
+Barnathan
+Barnar
+Barlip
+Barkins
+Barkenhagen
+Barkalow
+Barimah
+Baridon
+Barhydt
+Bargar
+Barff
+Bardeen
+Barcelona
+Barby
+Barbini
+Barbiere
+Barbetta
+Barberis
+Barberian
+Barban
+Barasch
+Baranow
+Baranovic
+Barajos
+Baraby
+Bapties
+Banyas
+Bantug
+Bantin
+Bantillan
+Bantay
+Bansbach
+Bankemper
+Banis
+Banick
+Banecker
+Bandin
+Bandemer
+Bandanza
+Bance
+Banales
+Bammon
+Bamfield
+Bambacigno
+Bambaci
+Balyeat
+Balvanz
+Balsano
+Balmores
+Ballreich
+Balloon
+Ballmer
+Ballintyn
+Balley
+Balletta
+Balhorn
+Balford
+Balezentis
+Baldrey
+Baldiviez
+Balder
+Baldassarre
+Baldacchino
+Balchunas
+Balceiro
+Balbin
+Balaz
+Balaski
+Balancia
+Balagtas
+Bakst
+Bakkum
+Bakios
+Bakeley
+Bajorek
+Bajdas
+Baizer
+Baitg
+Baise
+Bailony
+Baillio
+Baille
+Baiera
+Bahun
+Bah
+Bagne
+Bagi
+Baghdasarian
+Bageant
+Bagdonas
+Baetz
+Baeringer
+Badget
+Badeau
+Baddeley
+Bacy
+Backey
+Backenstose
+Backen
+Backe
+Backbone
+Baccouche
+Bacco
+Bacarella
+Babitsch
+Babena
+Babbin
+Babbel
+Babat
+Bab
+Azzaro
+Azoulay
+Azimi
+Azer
+Aylsworth
+Ayarza
+Axline
+Axelsen
+Awtrey
+Avola
+Avie
+Avetisyan
+Averyt
+Aveado
+Avanzato
+Avala
+Auyer
+Auxilien
+Auwarter
+Aurges
+Aures
+Auprey
+Aupperle
+Aunkst
+Aumich
+Aument
+Aumavae
+Aulbach
+Aukes
+Augspurger
+Auffrey
+Attridge
+Attkisson
+Attinger
+Atta
+Aton
+Atoe
+Atiyeh
+Athmann
+Athay
+Atchity
+Atallah
+Atala
+Astwood
+Astolfi
+Astol
+Asters
+Aspegren
+Asma
+Ashpole
+Ashfield
+Ashely
+Asevedo
+Aschmann
+Asar
+Asaeli
+Arzilli
+Arundel
+Arujo
+Aruiso
+Arturo
+Artry
+Artison
+Artinian
+Arrizaga
+Arriazola
+Arpino
+Arons
+Aronhalt
+Arntt
+Arniotes
+Arnholtz
+Arneberg
+Armillei
+Armijos
+Arm
+Arleth
+Arlen
+Arlan
+Arkins
+Arjes
+Arizzi
+Arizola
+Ariyoshi
+Aring
+Arimoto
+Arigo
+Arietta
+Arie
+Aridas
+Aricas
+Arhelger
+Arhart
+Arguillo
+Arguellez
+Argote
+Argenal
+Arenos
+Arenivas
+Arenivar
+Arendz
+Arendsee
+Arebela
+Ardizzone
+Ardion
+Ardery
+Ardd
+Ardan
+Arcino
+Arcilla
+Arcea
+Arcaute
+Arcangel
+Arcadipane
+Arbry
+Araque
+Aramini
+Arambuia
+Aragus
+Aragundi
+Aragoni
+Aragaki
+Aradanas
+Arabie
+Arabia
+Ar
+Apyuan
+Apuzzi
+Apruzzese
+Applewhaite
+Applebury
+Appeling
+Appelgate
+Apling
+Apking
+Apela
+Aparo
+Apa
+Aoay
+Anyan
+Antrican
+Antonopoulos
+Antonis
+Antonich
+Antonaccio
+Antona
+Antolik
+Antinore
+Anteby
+Anslinger
+Ansbacher
+Ansara
+Annette
+Ankersen
+Anis
+Aniol
+Aningalan
+Aniello
+Anichini
+Anibal
+Angviano
+Anglum
+Angley
+Angerer
+Angeloro
+Angeloff
+Angelocci
+Anestos
+Anerton
+Anelli
+Andzulis
+Andruss
+Andrian
+Andreatta
+Andonian
+Andon
+Anderon
+Andebe
+Andary
+Ancy
+Ancell
+Anasagasti
+Anakalea
+Anagnostou
+Amyotte
+Amtower
+Amstein
+Amsinger
+Amsili
+Amphy
+Amonette
+Amolsch
+Amistoso
+Amisano
+Amidei
+Amesquieto
+Amert
+Amento
+Ameling
+Amelang
+Ambroz
+Ambrosone
+Ambres
+Amble
+Amberson
+Ambeau
+Amati
+Amargo
+Amancio
+Amailla
+Amadi
+Alzugaray
+Alvorez
+Alverest
+Alven
+Alvarengo
+Alvalle
+Alvacado
+Alummoottil
+Alukonis
+Alu
+Altwies
+Altum
+Altringer
+Altop
+Altheimer
+Altew
+Alterio
+Alsman
+Alsdon
+Alsbrooks
+Alsandor
+Alrich
+Alrais
+Almario
+Allor
+Allocca
+Allnutt
+Allmand
+Allhands
+Allgaeuer
+Allessi
+Allenbrand
+Allemond
+Allegre
+Allcorn
+Allbones
+Allamong
+Allaband
+Algeo
+Alge
+Alfreds
+Alfera
+Alexzander
+Alexiou
+Alexaki
+Alexader
+Alevedo
+Alerte
+Alekna
+Aleizar
+Alegi
+Alegar
+Aleff
+Alecca
+Aldrege
+Aldi
+Aldarondo
+Alcosiba
+Alcombright
+Alce
+Alcaoa
+Alcaide
+Albriton
+Albrekht
+Albracht
+Alberthal
+Alberro
+Alberda
+Alattar
+Alar
+Alampi
+Alamos
+Alaibilla
+Alacano
+Akuchie
+Akram
+Akinyooye
+Akiereisen
+Aimbez
+Ailstock
+Ahyou
+Ahrenholtz
+Ahonen
+Ahmau
+Ahlstedt
+Ahle
+Ahlborn
+Aharonof
+Aharon
+Ahal
+Aguino
+Aguillera
+Aguiler
+Agueda
+Aguallo
+Agrios
+Agriesti
+Agricola
+Agreste
+Agrela
+Agre
+Agney
+Agne
+Agliam
+Agerton
+Afoa
+Aflalo
+Affelt
+Affagato
+Afan
+Aemmer
+Adzhabakyan
+Ady
+Adside
+Adrovel
+Adrid
+Adonis
+Adleman
+Adle
+Adjutant
+Adesso
+Adels
+Addo
+Adamiak
+Acron
+Ackins
+Ackies
+Achziger
+Achzet
+Achekian
+Ache
+Acfalle
+Accetturo
+Abubakr
+Abson
+Abramowski
+Aboytes
+Aboulissan
+Abling
+Ablin
+Ablang
+Abke
+Abetrani
+Abernatha
+Abela
+Abeb
+Abdin
+Abdelwahed
+Abdella
+Abdeldayen
+Abdel
+Abbinanti
+Abbay
+Abbadessa
+Abaya
+Abaunza
+Abatti
+Aasby
+Aaland
+Aaby
+Zysett
+Zwinger
+Zweier
+Zuziak
+Zusman
+Zuro
+Zurkus
+Zurheide
+Zurawik
+Zuniega
+Zumot
+Zullig
+Zukowsky
+Zukof
+Zukerman
+Zuclich
+Zuchara
+Zubrzycki
+Zuberbuhler
+Zuazo
+Zsohar
+Zschoche
+Zrimsek
+Zoutte
+Zotos
+Zorzi
+Zoroiwchak
+Zorens
+Zoquier
+Zonia
+Zone
+Zondlo
+Zomora
+Zombro
+Zombory
+Zombo
+Zomberg
+Zolman
+Zollar
+Zolinski
+Zolinas
+Zoellick
+Zoelle
+Zoebisch
+Zodrow
+Zoda
+Zobell
+Zmiejko
+Zlotnick
+Zlatkin
+Ziyad
+Ziter
+Zita
+Zissler
+Zisser
+Zirin
+Zircher
+Zipse
+Zipkin
+Zipay
+Zinni
+Zinkl
+Zimit
+Zimba
+Ziman
+Ziler
+Zilahi
+Ziko
+Zihal
+Zieske
+Zieser
+Zientara
+Ziencina
+Zielonko
+Ziek
+Ziehm
+Ziego
+Ziegenhagen
+Ziedan
+Ziebold
+Zidzik
+Zickuhr
+Zicari
+Zibert
+Zibelli
+Ziak
+Ziadie
+Zezima
+Zeyadeh
+Zeto
+Zetes
+Zerzan
+Zerring
+Zerom
+Zerck
+Zerbel
+Zentgraf
+Zenker
+Zener
+Zenbaver
+Zena
+Zemon
+Zemjanis
+Zeminski
+Zelmar
+Zellous
+Zellefrow
+Zelkind
+Zeleny
+Zelenko
+Zeis
+Zeimetz
+Zeimantz
+Zeilman
+Zehnpfennig
+Zehe
+Zeegers
+Zeckzer
+Zebell
+Zebel
+Zeals
+Zdrojkowski
+Zazozdor
+Zaxas
+Zawadzki
+Zavatson
+Zavadoski
+Zatko
+Zastawny
+Zaspel
+Zarzuela
+Zarycki
+Zarucki
+Zart
+Zarriello
+Zarozinski
+Zarnick
+Zarkin
+Zaritsky
+Zarella
+Zappolo
+Zappile
+Zappavigna
+Zapoticky
+Zapico
+Zapato
+Zapatas
+Zanueta
+Zanter
+Zanola
+Zanis
+Zaneski
+Zanco
+Zamzam
+Zamperini
+Zamparini
+Zampaglione
+Zamostny
+Zammiello
+Zammetti
+Zambotti
+Zamborsky
+Zam
+Zalwsky
+Zakarian
+Zaituna
+Zaitlin
+Zaidel
+Zaic
+Zaibel
+Zahri
+Zahradka
+Zahra
+Zahorchak
+Zaharchuk
+Zagorac
+Zagen
+Zaffina
+Zaffalon
+Zadra
+Zadow
+Zador
+Zadd
+Zacharia
+Zacharewicz
+Zablonski
+Zabka
+Zabik
+Zabielski
+Zabek
+Yuzn
+Yuste
+Yusi
+Yurkanin
+Yurich
+Yurchiak
+Yungclas
+Yungbluth
+Yunan
+Yuki
+Yueh
+Yucha
+Yslava
+Yrigollen
+Yragui
+Ypina
+Yozamp
+Yovino
+Yovanovich
+Yournet
+Younkins
+Younglove
+Younglas
+Youket
+Yosko
+Yoshimori
+Yorton
+Yorn
+Yorkman
+Yorio
+Yorgey
+Yoquelet
+Yonkoske
+Yongue
+Yonge
+Yoney
+Yonemori
+Yonek
+Yokiel
+Yokely
+Yoders
+Yo
+Yngsdal
+Ylonen
+Yilma
+Yidiaris
+Yezek
+Yestramski
+Yessios
+Yeskey
+Yerry
+Yerly
+Yerbich
+Yenz
+Yenney
+Yenner
+Yenglin
+Yengich
+Yendell
+Yeldon
+Yekel
+Yeisley
+Yeilding
+Yegge
+Yeend
+Yeeloy
+Yearicks
+Yeamans
+Yeakle
+Ydara
+Ybos
+Yballe
+Yavorsky
+Yater
+Yasutomi
+Yasinski
+Yarzabal
+Yarrell
+Yarish
+Yanoff
+Yannotti
+Yankovitz
+Yanity
+Yanetta
+Yandura
+Yancik
+Yanan
+Yanai
+Yamnitz
+Yammine
+Yamkosumpa
+Yakulis
+Yaklich
+Yakel
+Yahraus
+Yahna
+Yahl
+Yagoudaef
+Yagin
+Yagecic
+Yaftali
+Yafei
+Yafai
+Yablonsky
+Xander
+Wzorek
+Wykes
+Wydryck
+Wydo
+Wydler
+Wycuff
+Wyborny
+Wurts
+Wurgler
+Wuolle
+Wunderly
+Wun
+Wulkan
+Wuitschick
+Wuestenberg
+Wuerz
+Wuellenweber
+Wucherer
+Wublin
+Wubbel
+Wrotten
+Wrinkles
+Wriedt
+Wrenne
+Wreede
+Wraggs
+Woyahn
+Woulard
+Woudenberg
+Woskobojnik
+Wosher
+Wortinger
+Worstell
+Worst
+Worner
+Worn
+Wormely
+Worlow
+Workings
+Workinger
+Wootan
+Woolhouse
+Wooleyhan
+Woolcott
+Woodliff
+Woodert
+Woodend
+Woodburg
+Woodand
+Women
+Wombolt
+Wolzen
+Wolthuis
+Wolsted
+Wolsky
+Woloszczak
+Woller
+Wolkowski
+Wolkowiecki
+Woliver
+Wolhok
+Wolfsberger
+Wolfred
+Wolffe
+Wolfertz
+Wolbeck
+Wokwicz
+Wojtowich
+Wojtecki
+Wojnaroski
+Wojeik
+Woiwode
+Wohlwendi
+Wohlschlegel
+Wohlrab
+Wohld
+Woester
+Woernle
+Woelzlein
+Woelfle
+Wodskow
+Wlosinski
+Wlodyka
+Wlazlowski
+Wlach
+Wizar
+Wiuff
+Witvoet
+Wittstruck
+Wittry
+Wittliff
+Witterstauter
+Witsell
+Witosky
+Withy
+Witherbee
+Withenshaw
+Witczak
+Wisterman
+Wisnosky
+Wisniowski
+Wiskowski
+Wisk
+Wisinger
+Wisenor
+Wischner
+Wisbey
+Wirtjes
+Wirght
+Wirf
+Wipprecht
+Winzler
+Winzenried
+Wintringham
+Winterton
+Winterfeldt
+Winterbottom
+Winsted
+Wins
+Winninger
+Winning
+Winney
+Winnewisser
+Winners
+Winnegan
+Winklepleck
+Winkleblack
+Winkelpleck
+Winkeljohn
+Winkelbauer
+Winingear
+Winikoff
+Wingstrom
+Winett
+Winesickle
+Winesberry
+Winek
+Windmeyer
+Windhurst
+Windam
+Wimpey
+Wiman
+Wilts
+Wiltjer
+Wilterdink
+Willrett
+Willour
+Willmes
+Willmann
+Willinsky
+Willington
+Willigar
+Williama
+Willegal
+Willcoxon
+Willand
+Willame
+Willaby
+Wilkowitz
+Wilkers
+Wilison
+Wilis
+Wilgocki
+Wilging
+Wilfinger
+Wilebski
+Wildin
+Wildfong
+Wilderson
+Wildenthaler
+Wildeisen
+Wildauer
+Wilcinski
+Wilansky
+Wilabay
+Wikins
+Wikert
+Wik
+Wiinikainen
+Wiggains
+Wigen
+Wieto
+Wiess
+Wiesman
+Wierzba
+Wierschen
+Wierschem
+Wiehe
+Wieger
+Wiederwax
+Wiederin
+Wiede
+Wieciech
+Wiechert
+Wiechec
+Widrig
+Widowski
+Widmaier
+Widlak
+Widdoes
+Wickus
+Wicketts
+Wickemeyer
+Wicka
+Wicinsky
+Wibeto
+Wibberley
+Wibbenmeyer
+Wiatrak
+Wiatr
+Wiand
+Whyman
+Wholly
+Whittley
+Whittiker
+Whitteker
+Whitset
+Whitmyre
+Whitmeyer
+Whitheld
+Whitesinger
+Whitemore
+Whitacker
+Whistle
+Whisker
+Whisenton
+Whippie
+Whipp
+Whildin
+Whigum
+Whiby
+Whelton
+Wheeington
+Whan
+Whaler
+Whal
+Weyhrauch
+Wewerka
+Wetterauer
+Wetselline
+Wetklow
+Westwater
+Westrom
+Westre
+Westhouse
+Westervoorde
+Westergaard
+Westerbeck
+Westcote
+Westaway
+Wesselink
+Wesselhoft
+Weslowski
+Weslow
+Wescovich
+Werthman
+Wershey
+Werries
+Wernli
+Werning
+Werma
+Werking
+Wenzell
+Wentzloff
+Wentcell
+Wenstrand
+Wensky
+Wennersten
+Wenman
+Wengren
+Wener
+Weneck
+Wendy
+Wendte
+Wenderoth
+Wend
+Wenclawiak
+Wence
+Wemark
+Weltmer
+Welms
+Welman
+Wellendorf
+Welfel
+Weitkamp
+Weith
+Weiszbrod
+Weissmann
+Weissert
+Weisse
+Weissbrodt
+Weismiller
+Weisiger
+Weisenhorn
+Weisenfluh
+Weisend
+Weisenberg
+Weisdorfer
+Weisberger
+Weirather
+Weinzinger
+Weinzimer
+Weinzetl
+Weintz
+Weinand
+Weiker
+Weikal
+Weik
+Weigman
+Weigleb
+Weigart
+Weidenheimer
+Weiden
+Weickum
+Wehring
+Wehausen
+Weglin
+Weghorst
+Weeth
+Weeter
+Weenum
+Weelborg
+Weegar
+Weeber
+Wedwick
+Wedner
+Wedlow
+Wedlock
+Wedi
+Wedgworth
+Weckenborg
+Wechselblatt
+Webbs
+Webbink
+Weavil
+Weatherley
+Weatherill
+Wearrien
+Wearly
+Weagel
+Weadon
+Waymer
+Wayde
+Waybill
+Wavra
+Waughtel
+Waughtal
+Wauch
+Watzke
+Wattson
+Watrs
+Watral
+Watne
+Waterston
+Waszmer
+Wasylow
+Wasyliszyn
+Wassermann
+Wassenberg
+Wassenaar
+Waskow
+Waskey
+Waska
+Washurn
+Washup
+Washuk
+Washnock
+Washman
+Washinski
+Wasem
+Wartman
+Warsme
+Warsing
+Warschaw
+Warsager
+Warpool
+Warneka
+Warnasch
+Warmbier
+Warley
+Warick
+Warholic
+Warhola
+Warhol
+Warens
+Wareheim
+Wardrop
+Wardon
+Wardman
+Wardinsky
+Wardian
+Wappel
+Wanvig
+Wanser
+Wanschek
+Wanland
+Waninger
+Wanders
+Wampol
+Walzier
+Walvoord
+Walto
+Waltenbaugh
+Waltemath
+Waloven
+Walman
+Wally
+Wallravin
+Wallor
+Wallinga
+Walles
+Wallentine
+Wallenda
+Walleck
+Wallbrown
+Wallberg
+Wallbank
+Walland
+Wallaker
+Wallaert
+Wallack
+Walkinshaw
+Walking
+Walicki
+Waldrope
+Waldmann
+Waldenberg
+Walczynski
+Walchli
+Walbrecht
+Wakula
+Wakham
+Wakenight
+Wakeling
+Waitkus
+Waisman
+Waisath
+Wainman
+Wahoske
+Wahner
+Wahlenmaier
+Wahid
+Wagon
+Waggaman
+Wagenheim
+Waganer
+Wafula
+Waeyaert
+Waetzig
+Waelti
+Waeckerlin
+Waddouds
+Wackman
+Wackerbarth
+Wachsmuth
+Wabasha
+Vyhnal
+Vuturo
+Vulgamott
+Vukich
+Vrias
+Vranich
+Vrablic
+Votraw
+Voter
+Votaua
+Voskowsky
+Vorwaller
+Vorholt
+Voracek
+Voong
+Vonwagoner
+Vonstaden
+Vonsoosten
+Vonkrosigk
+Vongxay
+Vongvivath
+Vongunten
+Vongsakda
+Vongal
+Vonfeldt
+Vondohlen
+Vonderkell
+Vonbraunsberg
+Vonarx
+Volpert
+Volper
+Volpa
+Volmink
+Vollmering
+Volking
+Volkers
+Volkens
+Volin
+Volesky
+Volckmann
+Vojta
+Voita
+Voights
+Vogtman
+Vogtlin
+Voglund
+Vogland
+Vogenthaler
+Vogelpohl
+Vogds
+Voetmann
+Voedisch
+Vodder
+Voce
+Vlk
+Vlasaty
+Vlasak
+Vlahovich
+Vizza
+Vizuete
+Vivolo
+Vittum
+Vittek
+Vitorino
+Vitkus
+Vititow
+Vitera
+Vitantonio
+Vitaniemi
+Visvardis
+Vissman
+Visovsky
+Visosky
+Visocsky
+Visnosky
+Visnocky
+Viscarro
+Visaya
+Virts
+Virkler
+Virgili
+Virgie
+Virgel
+Virelli
+Viramontas
+Viorel
+Vintinner
+Vintimilla
+Vinsel
+Viniegra
+Vinck
+Villot
+Villenas
+Villemarette
+Villecus
+Villaquiran
+Villane
+Villalouos
+Villaescusa
+Vilkoski
+Vilkama
+Vilca
+Vilaro
+Vilardo
+Vilandre
+Viken
+Vigus
+Viguerie
+Vigorito
+Vigario
+Viessman
+Viesselman
+Viesca
+Vierthaler
+Vierps
+Vientos
+Vienneau
+Vidler
+Victorica
+Vickey
+Vicioso
+Vichidvongsa
+Viccica
+Veysey
+Vespia
+Veselic
+Verzi
+Versele
+Veroba
+Vernet
+Verlotte
+Verigan
+Verhaag
+Vergamini
+Verga
+Verfaille
+Verela
+Vere
+Verdine
+Verdiguel
+Verd
+Verbridge
+Verble
+Verbit
+Verbilla
+Verbasco
+Ventur
+Ventrice
+Ventre
+Ventors
+Venth
+Venosh
+Vennari
+Venkus
+Veninga
+Venible
+Venghaus
+Venetos
+Venere
+Veneable
+Vendelin
+Vemura
+Velzeboer
+Veltre
+Veltin
+Veloso
+Veles
+Vele
+Veld
+Veitz
+Veitenheimer
+Vein
+Veillette
+Vegher
+Vegetabile
+Vegar
+Veerkamp
+Veen
+Vecino
+Vebel
+Veater
+Veader
+Ve
+Vayon
+Vayner
+Vavricek
+Vauter
+Vaulx
+Vaughner
+Vaudreuil
+Vaubel
+Vattikuti
+Vathroder
+Vatch
+Vastola
+Vastardis
+Vassure
+Vassil
+Vassie
+Vasseur
+Vassen
+Vasquiz
+Vasaure
+Varvil
+Vartanyan
+Varron
+Varro
+Vargis
+Varesko
+Varda
+Varanese
+Varakuta
+Varagona
+Vanzante
+Vanyo
+Vanwyngaarden
+Vanwassenhove
+Vanvolkenburg
+Vanvalen
+Vantuyl
+Vantil
+Vanta
+Vanstrom
+Vanslooten
+Vansicklin
+Vanscoik
+Vanschaick
+Vanruiten
+Vanostberg
+Vanorsdol
+Vanolinda
+Vanoflen
+Vannuland
+Vannover
+Vannorsdell
+Vanniello
+Vanni
+Vanner
+Vanmarter
+Vanleuvan
+Vanlaar
+Vankilsdonk
+Vankammen
+Vanhevel
+Vanheukelem
+Vanhee
+Vanhauen
+Vanhamlin
+Vanhamersveld
+Vangyi
+Vangompel
+Vangoff
+Vangerbig
+Vangelos
+Vanfossan
+Vanez
+Vaneffen
+Vandygriff
+Vandy
+Vanduynhoven
+Vandunk
+Vandorien
+Vandon
+Vandiest
+Vandeweert
+Vandevort
+Vandevere
+Vandeveble
+Vandestreek
+Vandesteeg
+Vanderwyk
+Vanderwood
+Vanderwilt
+Vanderwege
+Vanderweerd
+Vanderweel
+Vandertuig
+Vanderstappen
+Vanderschoot
+Vandermoon
+Vanderkaaden
+Vanderhoot
+Vanderboom
+Vanderau
+Vandenacre
+Vandemortel
+Vandeman
+Vandelaare
+Vandebrake
+Vanconant
+Vancleaf
+Vanbogelen
+Vanbenthuyse
+Vanbeck
+Vanasselt
+Vanaprasert
+Vanandel
+Vampa
+Valseca
+Valree
+Valot
+Valorie
+Vallimont
+Vallie
+Vallentine
+Vallelonga
+Vallario
+Vall
+Valgren
+Valer
+Valenzvela
+Valentyn
+Valenstein
+Valenciana
+Valderamo
+Valcin
+Valcho
+Valakas
+Vaksman
+Vakil
+Vaka
+Vajgrt
+Vaissiere
+Vainio
+Vaiko
+Vaghy
+Vaghn
+Vafiadis
+Vafiades
+Vaeza
+Vaeth
+Vadasy
+Vaclavik
+Vacio
+Vaci
+Vache
+Vaccarino
+Vacante
+Uzun
+Uxa
+Uvalles
+Utvik
+Uttley
+Ustico
+Usman
+Usina
+Ushioda
+Ushijima
+Uscio
+Usack
+Urse
+Urrey
+Urreta
+Urraca
+Urness
+Urlanza
+Uriostejue
+Urik
+Urenio
+Urdiano
+Urbieta
+Uptegraft
+Uppencamp
+Unterkofler
+Unnold
+Unnewehr
+Unkn
+Uniacke
+Unglaub
+Unck
+Umnus
+Umezawa
+Umbel
+Ultseh
+Ultreras
+Ulses
+Ullum
+Ulisch
+Ulicnik
+Ulich
+Uleman
+Ukich
+Uken
+Uhrin
+Uhrhammer
+Uhles
+Uhlenhopp
+Ugaz
+Ugaitafa
+Ueki
+Uebersax
+Udinsky
+Udicious
+Ucha
+Uccio
+Uc
+Ubry
+Ubiles
+Ubertini
+Ubence
+Tyssens
+Tysseling
+Tyrance
+Tynio
+Tylman
+Tydings
+Tydeman
+Twohatchet
+Twito
+Twillie
+Twiet
+Twiest
+Tweet
+Tweddell
+Twait
+Tvedt
+Tuxbury
+Tuukanen
+Tutuska
+Tutoni
+Tutela
+Tushoski
+Turvaville
+Turturo
+Turrill
+Turrie
+Turpiano
+Turomsha
+Turocy
+Turnpaugh
+Turnow
+Turnmyre
+Turnier
+Turkmay
+Turkasz
+Turinetti
+Tureson
+Turdo
+Turcio
+Turbiner
+Turbide
+Turber
+Turbe
+Turansky
+Tupy
+Tuppen
+Tuplano
+Tuorto
+Tunon
+Tunget
+Tunby
+Tun
+Tumolillo
+Tumminia
+Tumbleston
+Tullison
+Tulis
+Tuliau
+Tukuafa
+Tukis
+Tujague
+Tuia
+Tugade
+Tuffin
+Tuesburg
+Tuerk
+Tuer
+Tuenge
+Tudruj
+Tudman
+Tudisco
+Tuccio
+Tucay
+Tuberman
+Tsuruda
+Tsuchiura
+Tsuchida
+Tsistinas
+Tshudy
+Tschirhart
+Tschache
+Tsantakis
+Trzaska
+Trythall
+Tryninewski
+Truont
+Trumpp
+Truka
+Truiolo
+Truglio
+Trueluck
+Trudo
+Truchon
+Trucchio
+Trube
+Truan
+Troxil
+Trowel
+Trovinger
+Trotz
+Trotto
+Trosen
+Troost
+Tronzo
+Tront
+Trometter
+Trombino
+Tromba
+Trollope
+Troke
+Trojanovich
+Trojak
+Trohanov
+Trogstad
+Troe
+Trocchio
+Trobridge
+Trobough
+Trnong
+Trivane
+Trippel
+Trimnal
+Trimis
+Trimino
+Trilt
+Trillas
+Trillana
+Triglia
+Trigillo
+Trifone
+Triffo
+Trifero
+Tridenti
+Tricoli
+Tricamo
+Tribue
+Triblett
+Trevithick
+Trevisone
+Trevis
+Trevillian
+Trevethan
+Treves
+Treusdell
+Tretola
+Tretina
+Tretera
+Tressel
+Treola
+Trentz
+Trento
+Trentman
+Trenor
+Trennell
+Trend
+Trenchard
+Tremore
+Tremillo
+Trembinski
+Trelles
+Treister
+Treine
+Treible
+Treff
+Tredinnick
+Treder
+Trebon
+Trebesch
+Trear
+Traviss
+Traux
+Trautner
+Trausch
+Traum
+Trattner
+Trass
+Traphagen
+Trapeni
+Trapalis
+Traner
+Tramonti
+Trainham
+Traicoff
+Trahern
+Traffanstedt
+Trachsel
+Tracewell
+Trabold
+Trabazo
+Tozloski
+Toyota
+Toyn
+Towse
+Townsand
+Towels
+Touton
+Toussand
+Toupe
+Touney
+Toudle
+Touchard
+Touby
+Touart
+Totzke
+Tototzintle
+Totino
+Toting
+Tossie
+Tosco
+Tosch
+Tortu
+Tortolano
+Tortelli
+Torruellas
+Torros
+Torrion
+Torrillo
+Torrico
+Torreblanca
+Torrano
+Torongeau
+Toromanides
+Tornincasa
+Torey
+Toren
+Torbus
+Toquinto
+Topolewski
+Topoian
+Topness
+Toplistky
+Topliffe
+Topal
+Topacio
+Toothacre
+Tooms
+Toolsiram
+Toolan
+Tookmanian
+Tonzi
+Tonti
+Tonschock
+Tonsall
+Tonrey
+Tonnesen
+Tonnar
+Tongate
+Tonetti
+Tonelson
+Tonder
+Tonai
+Tomspon
+Tomski
+Tomshack
+Tomkus
+Tomka
+Tomidy
+Tomichek
+Tomeldan
+Tomehak
+Tombleson
+Tomasson
+Tomasic
+Tomash
+Tomanek
+Tolontino
+Tollin
+Tollerud
+Tollefsen
+Toline
+Tokley
+Tokkesdal
+Tohen
+Togashi
+Tofolla
+Toepperwein
+Toeller
+Toelke
+Toedebusch
+Todt
+Todoroff
+Todor
+Todesco
+Toboz
+Tobolski
+Toaston
+Toa
+Tlumacki
+Tlatenchi
+Tlatelpa
+Tlamka
+Tjandra
+Tix
+Tivis
+Tivar
+Titterness
+Titone
+Titler
+Tith
+Tisi
+Tish
+Tisdel
+Tisdal
+Tischner
+Tipre
+Tippey
+Tipold
+Tinucci
+Tintinger
+Tinnerello
+Tinn
+Tinlin
+Tinger
+Timus
+Timothe
+Timons
+Timonere
+Timon
+Timenez
+Timchula
+Timbrell
+Timas
+Timar
+Tilzer
+Tilus
+Tilt
+Tilow
+Tillou
+Tietge
+Tieng
+Tichnell
+Tichi
+Tibor
+Thy
+Thury
+Thurness
+Thurlby
+Thurby
+Thuney
+Thuma
+Thull
+Thruthley
+Throssell
+Thress
+Threlfall
+Thrapp
+Thrams
+Thraen
+Thouvenel
+Thorstenson
+Thorsness
+Thoroughgood
+Thornborough
+Thormaehlen
+Thorade
+Thonney
+Thompon
+Thometz
+Thomeczek
+Thomases
+Thomae
+Thoburn
+Thobbs
+Thivener
+Thim
+Thilmony
+Thiengtham
+Thielges
+Thieklin
+Thidphy
+Thibaut
+Thibadeau
+Thew
+Theule
+Theuenin
+Thepbanthao
+Theos
+Thell
+Thelin
+Thelemaque
+Theinert
+Theeman
+Theden
+Thebo
+Thansamai
+Thanos
+Thangavelu
+Thanem
+Thanasouk
+Thanas
+Thamann
+Thaman
+Thalls
+Thaller
+Thall
+Thadison
+Tewolde
+Tewa
+Teuteberg
+Teteak
+Testolin
+Tessendorf
+Tess
+Tesmar
+Teschler
+Terwey
+Tertinek
+Terstage
+Terrone
+Terrible
+Terrian
+Terrezza
+Terracciano
+Terp
+Teroganesyan
+Termilus
+Terinoni
+Teri
+Terhorst
+Terherst
+Terazes
+Teravainen
+Teque
+Teoh
+Teodoro
+Tention
+Tenore
+Tenofsky
+Tenn
+Tenhoff
+Tenhaeff
+Tengben
+Tenerovich
+Tener
+Tenda
+Tenario
+Tempelton
+Temoney
+Teman
+Tellefsen
+Telkamp
+Telgen
+Teles
+Telch
+Telander
+Teklu
+Teixeria
+Teissedre
+Teisberg
+Tehney
+Tegner
+Tegan
+Teehee
+Teder
+Teddy
+Tecuanhuey
+Techau
+Tecchio
+Teakell
+Teager
+Taylar
+Tayan
+Tawwab
+Tavolieri
+Taverab
+Tavaris
+Tavana
+Tauzin
+Tautolo
+Tausch
+Taula
+Taualii
+Tattrie
+Tatsuhara
+Taton
+Tatge
+Tatel
+Tastet
+Tassa
+Tasma
+Taskey
+Tashiro
+Taruer
+Taruc
+Tartsah
+Tarski
+Tarrenis
+Tarnoff
+Tarmey
+Tarman
+Tarling
+Tarella
+Tarduno
+Tarboro
+Tarbert
+Taray
+Taras
+Taque
+Tapian
+Taphous
+Tapaoan
+Tanzi
+Tantum
+Tannous
+Tankxley
+Tankesly
+Tanh
+Tangney
+Tangerman
+Tangaro
+Tangari
+Tangabekyan
+Tandus
+Tande
+Tamkin
+Tami
+Tamburrelli
+Tamburino
+Tamborlane
+Tamai
+Talvy
+Talsky
+Talleut
+Tallacksen
+Taliferro
+Talicska
+Talentino
+Talaro
+Talamentez
+Talaga
+Tako
+Taker
+Takara
+Takai
+Tajudeen
+Tajima
+Taitague
+Taillefer
+Tail
+Tahon
+Tagupa
+Taglauer
+Tagalog
+Tagaloe
+Tagala
+Tagaca
+Tag
+Tafiti
+Tafelski
+Taetzsch
+Taegel
+Tadt
+Tadgerson
+Taddio
+Tadd
+Tacopino
+Tacneau
+Tackette
+Tackes
+Tacke
+Tachauer
+Tacason
+Tabuena
+Tabion
+Tabatt
+Szysh
+Szymonik
+Szwede
+Szulimowski
+Szpak
+Szoka
+Szocki
+Szklarski
+Szitar
+Szewc
+Szesterniak
+Szermer
+Szerbin
+Szczepkowski
+Szczeblewski
+Szachewicz
+Szabat
+Syzdek
+Syrrakos
+Syria
+Sypult
+Sypolt
+Synovic
+Syner
+Symkowick
+Symeon
+Sylney
+Sylla
+Syktich
+Syer
+Swopshire
+Swolley
+Swithenbank
+Swiss
+Swirczek
+Swingler
+Swingen
+Swinerton
+Swinea
+Swille
+Swierenga
+Swierczynski
+Swieca
+Swicord
+Swerdloff
+Swenceski
+Swelt
+Swelgart
+Swehla
+Sweets
+Sweem
+Swed
+Sweatmon
+Sweatfield
+Swatman
+Swartzman
+Swartzell
+Swantak
+Swanston
+Swancutt
+Swanay
+Swamm
+Swam
+Swait
+Swainey
+Swaggart
+Swabe
+Swabb
+Svobodny
+Svetlak
+Svennungsen
+Svedine
+Svatos
+Svare
+Svancara
+Suydan
+Suwannakintho
+Suvada
+Suttin
+Suttee
+Sutkus
+Sutic
+Suthers
+Sutcliff
+Suszynski
+Sustar
+Sustaire
+Suskay
+Susany
+Susanin
+Suryanarayana
+Survis
+Surpris
+Suro
+Surminec
+Surguy
+Surgoine
+Sures
+Suren
+Surbella
+Suomela
+Sunyich
+Sunniga
+Sunier
+Sumrow
+Sumption
+Summerlot
+Sumerix
+Sumeriski
+Sultani
+Sulley
+Sullenberger
+Sulipizio
+Sulin
+Sulima
+Sulikowski
+Sulentic
+Sulejmanovski
+Sugabo
+Suffield
+Suentenfuss
+Suehs
+Sudekum
+Sudbrock
+Sucre
+Suchocki
+Suchla
+Sucgang
+Succar
+Subijano
+Subich
+Subert
+Subera
+Suaava
+Stuttgen
+Sturner
+Sturk
+Sturgul
+Sturghill
+Stukowski
+Stuesse
+Stuermer
+Stuer
+Stuebe
+Studyvance
+Studnicki
+Studniarz
+Studmire
+Studdiford
+Stucke
+Stublaski
+Stubby
+Stubbendeck
+Strzalkowski
+Struzzi
+Struzik
+Strubel
+Strozewski
+Strowe
+Strous
+Strotz
+Strombeck
+Stroker
+Strohmayer
+Strogen
+Strizich
+Strini
+Stringari
+Strimling
+Strimback
+Strife
+Strid
+Stricklind
+Stribley
+Strevels
+Strevell
+Streva
+Stretz
+Strenge
+Stremi
+Strelecki
+Strejan
+Streitnatter
+Streff
+Strefeler
+Streeton
+Stred
+Strazisar
+Strayhand
+Strayham
+Stravinski
+Strausz
+Strausner
+Strauhal
+Straugh
+Strasters
+Stranford
+Strandburg
+Stranahan
+Strahin
+Stradtner
+Stracquatanio
+Strachman
+Straathof
+Stpierrie
+Stoviak
+Stovell
+Stoutenger
+Stoudymire
+Stoud
+Stouch
+Stouall
+Stottlar
+Stotko
+Stothard
+Stotesbury
+Stotesberry
+Storto
+Stores
+Storage
+Stoos
+Stonich
+Stolzenburg
+Stolly
+Stolebarger
+Stolcals
+Stolar
+Stoklasa
+Stogden
+Stoffey
+Stofferan
+Stoey
+Stoett
+Stoeltzing
+Stoel
+Stoeke
+Stoeffler
+Stoeckert
+Stoebner
+Stoeberl
+Stodomingo
+Stodder
+Stockwin
+Stockon
+Stocki
+Stockebrand
+Stocco
+Stobie
+Stlouise
+Stives
+Stirn
+Stire
+Stipanuk
+Stingle
+Stinespring
+Stinehour
+Stinebuck
+Stindt
+Stimple
+Stimler
+Stilwagen
+Stiltz
+Stilner
+Stillie
+Stigsell
+Stiern
+Stiens
+Stiehm
+Stiegman
+Stiegemeier
+Stieb
+Stidstone
+Sticklin
+Sticklen
+Stickford
+Sthole
+Stford
+Stflorant
+Steury
+Stetzenbach
+Stetke
+Sterpka
+Sterker
+Sterkenburg
+Sterkel
+Stephensen
+Stepan
+Step
+Stenz
+Stenn
+Stendeback
+Stenbeck
+Stenback
+Sten
+Stemmler
+Stelzl
+Steltzer
+Stellpflug
+Stellfox
+Stelk
+Stele
+Steinruck
+Steinmeiz
+Steinkuehler
+Steinkirchner
+Steinkellner
+Steinerkert
+Steine
+Steinbrink
+Steinbauer
+Steik
+Steighner
+Steiert
+Steich
+Steibel
+Stehno
+Steggeman
+Stefl
+Stefford
+Steffa
+Stefanatos
+Steep
+Steenwyk
+Steenhoven
+Steelmon
+Steeg
+Steeb
+Stedronsky
+Steczo
+Stecklair
+Stechuchak
+Stechlinski
+Steber
+Stebe
+Stearnes
+Stearne
+Stea
+Stdenny
+Stchur
+Stayter
+Stawicki
+Stavrositu
+Staudenmeier
+Stattelman
+Statires
+Station
+Stathos
+Stathas
+Stasulis
+Stassen
+Stasny
+Staser
+Staschke
+Starweather
+Stars
+Starnaud
+Starley
+Starkman
+Starken
+Starich
+Starghill
+Starcevic
+Staplins
+Stapelman
+Stanzak
+Stanway
+Stanowski
+Stankowitz
+Stankaitis
+Staniec
+Stania
+Stangroom
+Stanesic
+Stanert
+Staneart
+Stands
+Standors
+Standifur
+Standeven
+Standaert
+Stancoven
+Stanclift
+Stancey
+Stanbaugh
+Stana
+Stammler
+Stamenov
+Stambach
+Stamatopoulos
+Stamas
+Stalberger
+Stakoe
+Stakley
+Stakkeland
+Stakemann
+Stainbach
+Stagowski
+Stagno
+Stagman
+Stagles
+Stagers
+Staffeld
+Staenglen
+Staehler
+Stadther
+Stadt
+Stadnik
+Stadick
+Stachurski
+Stace
+Stabs
+Stabley
+Stable
+Srygley
+Srinvasan
+Squarciafico
+Squair
+Spyrakos
+Spyies
+Spycher
+Spurger
+Spulick
+Spudis
+Spuck
+Sprygada
+Spruiell
+Spruance
+Sprowls
+Sprouls
+Sprong
+Sprole
+Springe
+Sprewell
+Sprengelmeyer
+Sprawls
+Sprauve
+Spragley
+Spotorno
+Sporysz
+Sporman
+Sporich
+Spoonemore
+Spoleti
+Spohnholz
+Splitt
+Splett
+Splatt
+Spiter
+Spirounias
+Spirk
+Spire
+Spinoza
+Spinn
+Spinetti
+Spinello
+Spinar
+Spilis
+Spiliakos
+Spigutz
+Spielvogel
+Spicknall
+Spicker
+Sperier
+Speraw
+Spennicchia
+Spene
+Spellane
+Spegal
+Spee
+Specken
+Spearow
+Spearmon
+Spayd
+Spartin
+Spartichino
+Spart
+Sparacina
+Spannuth
+Spanner
+Spanicek
+Spanger
+Spane
+Spakes
+Spadard
+Spacht
+Spacagna
+Sozio
+Soyke
+Sowl
+Sowden
+Sowada
+Sovel
+Souvannakhily
+Souto
+Southand
+Sourlis
+Soulliere
+Souhrada
+Sou
+Sotos
+Sothen
+Sosbe
+Sorzano
+Sorvig
+Sortland
+Sorokata
+Soro
+Sorlie
+Sorhaindo
+Sorell
+Sordia
+Sorace
+Soptick
+Soppeland
+Sophy
+Sopczak
+Sooy
+Soop
+Soomaroo
+Soolua
+Sonterre
+Sonsteng
+Sonnefeld
+Sonnee
+Sonka
+Songy
+Sondrup
+Sondles
+Sondheimer
+Sonderman
+Sonderegger
+Somvang
+Somsy
+Somrak
+Somoza
+Somogye
+Somo
+Sommons
+Sommar
+Somji
+Somilleda
+Somerfield
+Somdah
+Somayor
+Solwold
+Solverud
+Soltow
+Soltmann
+Solow
+Solorsano
+Solonar
+Solomen
+Sollors
+Sollitto
+Solliday
+Solito
+Solinas
+Solima
+Solies
+Solien
+Solich
+Solian
+Solhjem
+Solera
+Soldeo
+Solazar
+Solarski
+Solaita
+Soladine
+Sokul
+Sokotowski
+Sokolski
+Sokolowich
+Sojo
+Soito
+Soiro
+Soifer
+Softich
+Sofer
+Soechting
+Sodini
+Sodervick
+Soders
+Sodawasser
+Sockey
+Sobrio
+Sobieraj
+Sobeski
+Sobery
+Soberanes
+Sobenes
+Sobe
+Sobanski
+Soape
+Snowder
+Snorden
+Snode
+Snetsinger
+Snaples
+Snaer
+Snaders
+Smyrski
+Smyntek
+Smykowski
+Smutzler
+Smutny
+Smulik
+Smugala
+Smuck
+Smolnicky
+Smolinsky
+Smitty
+Smithe
+Smiling
+Smiler
+Smigiel
+Smerdon
+Smeja
+Smedes
+Smeathers
+Smarra
+Smar
+Smallmon
+Smallin
+Smallidge
+Slyton
+Slutsky
+Sluski
+Slovinski
+Sloter
+Slonecker
+Slomer
+Slogeris
+Slobodnik
+Sloanes
+Slipper
+Slingluff
+Slingland
+Sliney
+Slimko
+Sliman
+Slimak
+Slessman
+Slepski
+Sleppy
+Sleiman
+Sleaford
+Slaugenhaupt
+Slark
+Slackman
+Slaboda
+Skyes
+Skweres
+Skwarek
+Skubik
+Skrzypinski
+Skrebes
+Skrabanek
+Skovlund
+Skotnicki
+Skone
+Skonczewski
+Skold
+Skoien
+Skoczen
+Skobiak
+Skimehorn
+Skillpa
+Skillett
+Skillan
+Skildum
+Skibski
+Skibo
+Skevofilakas
+Skepple
+Skarzynski
+Skartvedt
+Skar
+Skapura
+Skaflen
+Skaer
+Skabo
+Sjulstad
+Sjerven
+Sizar
+Sixt
+Sixsmith
+Siwicki
+Sivills
+Sivilay
+Sivie
+Sivick
+Sivay
+Sivalia
+Sival
+Siurek
+Siuda
+Sittre
+Sittner
+Sittman
+Sitterding
+Sitosky
+Sitkiewicz
+Sistek
+Sista
+Sisomphou
+Sisofo
+Sisley
+Siskin
+Sisavath
+Sirpilla
+Sirosky
+Sirolli
+Siroka
+Sirna
+Sirico
+Sirhan
+Siravo
+Sipriano
+Sippy
+Siphan
+Siona
+Siok
+Sinrich
+Sington
+Singharath
+Singewald
+Singerman
+Sinarath
+Simple
+Simper
+Simor
+Simoniello
+Simonetty
+Simonet
+Simokat
+Simoens
+Simmond
+Simmes
+Simitian
+Simich
+Simerson
+Simensky
+Simcock
+Silvestrini
+Silvaggio
+Siluis
+Siltman
+Silovich
+Sillitoe
+Silkenson
+Siliezar
+Silevinac
+Silence
+Silbiger
+Silao
+Sil
+Sikarskie
+Siglow
+Siglar
+Sifre
+Sifontes
+Sifers
+Sievertsen
+Sieverson
+Sieve
+Sietz
+Siert
+Sieradski
+Sier
+Sielaff
+Sieja
+Siedner
+Siedel
+Siebenthal
+Sidorowicz
+Sidley
+Sidi
+Sideman
+Sicks
+Sickel
+Sickafoose
+Sicinski
+Sibounma
+Sibgert
+Sibeto
+Sibel
+Sibal
+Siar
+Siaperas
+Siami
+Sialana
+Shyne
+Shybut
+Shwab
+Shutty
+Shutters
+Shusterman
+Shurr
+Shurak
+Shuptrine
+Shupert
+Shummon
+Shulthess
+Shult
+Shulse
+Shullick
+Shulick
+Shulenberger
+Shuffleburg
+Shubov
+Shry
+Shrigley
+Shren
+Shrawder
+Showen
+Shoulder
+Shorthair
+Shopbell
+Shoobridge
+Shongo
+Shoman
+Shollenbarger
+Shoji
+Shofestall
+Shodunke
+Shober
+Shivy
+Shisila
+Shirvanian
+Shirakawa
+Shippen
+Ship
+Shinsky
+Shinnick
+Shinkel
+Shingleur
+Shingledecker
+Shindel
+Shimon
+Shimaoka
+Shilo
+Shillito
+Shillingsford
+Shilkuski
+Shiliata
+Shildneck
+Shikuma
+Shike
+Shigeta
+Shigemi
+Shifferd
+Shider
+Shibi
+Shettleroe
+Shetterly
+Sherville
+Sherrock
+Sherrange
+Sherraden
+Sherles
+Sherief
+Sherbon
+Shepperdson
+Shenker
+Sheneman
+Shene
+Shempert
+Sheman
+Shelvy
+Shelsy
+Shelkoff
+Shekels
+Sheirich
+Sheingold
+Sheidler
+Shehee
+Shefte
+Sheftall
+Sheerer
+Sheer
+Sheakley
+Shbi
+Shawber
+Shatek
+Shasky
+Shary
+Sharplin
+Sharperson
+Sharabi
+Shappen
+Shapouri
+Shapleigh
+Shapino
+Shaper
+Shanno
+Shandro
+Shanberg
+Shamsi
+Shammah
+Shamir
+Shamily
+Shalwani
+Shalla
+Shaline
+Shalhoub
+Shakoor
+Shakin
+Shahinfar
+Shahin
+Shahim
+Shahbaz
+Shaffren
+Shaffen
+Shadfar
+Shadding
+Shadazz
+Shaben
+Shabel
+Sgueglia
+Sgrignoli
+Sgammato
+Seykoski
+Seyb
+Sewyerd
+Seweall
+Sewade
+Severi
+Seveney
+Sevadjian
+Settlemyre
+Settlemires
+Settino
+Settimo
+Setterland
+Seton
+Setler
+Setias
+Seti
+Setchell
+Setaro
+Sestoso
+Sessin
+Sesser
+Serville
+Servi
+Servedio
+Serve
+Serravalli
+Sermersheim
+Serfoss
+Serfling
+Serey
+Seres
+Serens
+Serene
+Sercovich
+Serban
+Seratti
+Seratt
+Serasio
+Serandos
+Seraiva
+Seraille
+Sepvlieda
+Sepulbeda
+Septelka
+Seppelt
+Seppanen
+Seppa
+Senz
+Senst
+Sensor
+Sensmeier
+Sensing
+Senseney
+Sensenbrenner
+Senseman
+Seniff
+Sengvilay
+Sengun
+Senethavilouk
+Senesenes
+Senderling
+Sender
+Senavanh
+Semsem
+Semonis
+Seminario
+Sember
+Selzler
+Selvester
+Selusi
+Selnes
+Sellin
+Sellards
+Selkey
+Selic
+Selgrade
+Selesnick
+Selakovic
+Seiters
+Seit
+Seisler
+Seil
+Seikaly
+Seidenbecker
+Seibt
+Seibers
+Seiavitch
+Segreto
+Segonia
+Seggerman
+Segerman
+Segelhorst
+Seferovic
+Sefcheck
+Seering
+Seemer
+Seekford
+Seekamp
+Seegar
+Seedorff
+Seedborg
+Seebaum
+Sedanos
+Secundo
+Second
+Seckletstewa
+Sechang
+Sebranek
+Sebion
+Sebero
+Sebeniecher
+Sebasovich
+Searer
+Seara
+Seanger
+Seajack
+Seaholtz
+Seagers
+Seaforth
+Seacrest
+Seacat
+Seaburn
+Sdoia
+Sczbecki
+Scurci
+Scullin
+Scuito
+Scudero
+Scucchi
+Scsarpisnato
+Scro
+Scrivener
+Scriuner
+Scripps
+Scrimsher
+Scrichfield
+Screnci
+Scrape
+Scouller
+Scotts
+Scotting
+Scorgie
+Scollan
+Sciullo
+Scites
+Scicutella
+Scialpi
+Sciacchitano
+Schy
+Schworm
+Schwizer
+Schwister
+Schwipps
+Schwertfeger
+Schwerdt
+Schwerd
+Schwenzer
+Schwenneker
+Schwendeman
+Schwemmer
+Schweitz
+Schwarzlose
+Schwart
+Schwantd
+Schwadron
+Schutze
+Schute
+Schusted
+Schurk
+Schumachor
+Schulter
+Schultens
+Schulkin
+Schulist
+Schuit
+Schuering
+Schueren
+Schueneman
+Schuemann
+Schuchat
+Schuber
+Schubach
+Schrumpf
+Schroot
+Schroen
+Schroedter
+Schreuder
+Schreacke
+Schrayter
+Schrawder
+Schrauger
+Schraub
+Schrameck
+Schraff
+Schradle
+Schrab
+Schowengerdt
+Schossow
+Schopmeyer
+Schopflin
+Schop
+Schomin
+Schomas
+Schomacker
+Scholtens
+Scholin
+Schoggen
+Schoessow
+Schoepfer
+Schoenmaker
+Schoenig
+Schoelman
+Schoellkopf
+Schoell
+Schoeben
+Schoderbek
+Schockley
+Schnure
+Schnorbus
+Schnopp
+Schnobrich
+Schnitz
+Schnickel
+Schnibbe
+Schnepf
+Schnelder
+Schneidman
+Schneeberger
+Schnackel
+Schmollinger
+Schmoak
+Schmittou
+Schmiot
+Schmille
+Schmier
+Schmiel
+Schmiedeskamp
+Schmidtka
+Schmidlin
+Schmertz
+Schmerge
+Schmerer
+Schmelmer
+Schmeidler
+Schmautz
+Schmauder
+Schmatz
+Schmand
+Schmaling
+Schlund
+Schlumaker
+Schlotthauer
+Schlotte
+Schlotfeldt
+Schlote
+Schlossman
+Schloemann
+Schlindwein
+Schlimmer
+Schlieter
+Schlichenmaye
+Schleppy
+Schlenger
+Schleker
+Schleibaum
+Schleh
+Schlecter
+Schlaefli
+Schladweiler
+Schlabs
+Schirrmacher
+Schiralli
+Schinnell
+Schinker
+Schingeck
+Schindewolf
+Schimel
+Schilsky
+Schilk
+Schilder
+Schifko
+Schiffmann
+Schierenbeck
+Schierbrock
+Schielke
+Schieferstein
+Schiefen
+Schickedanz
+Schey
+Scheuren
+Scheuers
+Scherschligt
+Scherma
+Scherbring
+Scherbel
+Scheno
+Schenfeld
+Schells
+Schellin
+Schellermann
+Scheiern
+Scheiderer
+Schegetz
+Scheffrahn
+Scheffert
+Schechinger
+Schavone
+Schaunt
+Schaumann
+Schauble
+Schaubhut
+Schatzle
+Scharmann
+Scharler
+Scharbrough
+Schap
+Schanzenbach
+Schantini
+Schange
+Schandel
+Schammel
+Schallig
+Schaffter
+Schaffeld
+Schaffel
+Schafersman
+Schaen
+Schachterle
+Schachsieck
+Schabbing
+Scelzo
+Scelsi
+Scavo
+Scavetta
+Scaturro
+Scatenato
+Scarpitto
+Scarpitta
+Scarpato
+Scarpati
+Scarp
+Scarlato
+Scargall
+Scarfi
+Scantlen
+Scanneu
+Scannapieco
+Scanio
+Scandrett
+Scandalios
+Scancarello
+Scamehorn
+Scalzi
+Scallorn
+Scallion
+Scalet
+Scaiano
+Scaia
+Scagliotti
+Scace
+Sboro
+Sbarra
+Saysongkham
+Saysana
+Sayloe
+Saxinger
+Saxfield
+Sawtell
+Sawransky
+Sawhill
+Sawatzki
+Sawaia
+Savitch
+Savinar
+Savi
+Saven
+Savas
+Savaria
+Savakis
+Sava
+Sauveur
+Sausser
+Saurey
+Sauredo
+Saunas
+Saulsbery
+Sauger
+Sauerhage
+Sauerbry
+Sauce
+Sauby
+Satz
+Sattlefield
+Satmary
+Sathiraboot
+Satchwell
+Sat
+Sasuille
+Sashington
+Sasengbong
+Sasao
+Sarwar
+Sarrell
+Sarraga
+Saroop
+Sarnes
+Sarnacki
+Sarlo
+Sarks
+Sarkodie
+Sark
+Sargis
+Sargetakis
+Saretto
+Sarette
+Sarensen
+Sarcinelli
+Sarcinella
+Sarcia
+Saras
+Saranzak
+Saraniti
+Sarani
+Sarafian
+Saraf
+Sarac
+Sarabando
+Saporita
+Sapnu
+Sapko
+Saous
+Sanzenbacher
+Santti
+Santrizos
+Santoscoy
+Santomauro
+Santolucito
+Santis
+Santio
+Santilukka
+Santaloci
+Santagata
+Santaella
+Sanseda
+Sanquenetti
+Sanots
+Sanosyan
+Sann
+Sanmarco
+Sanlatte
+Sankovich
+Sanke
+Sankary
+Sankaran
+Sanislo
+Sanipasi
+Saniger
+Sangren
+Sanghez
+Saneaux
+Sandstedt
+Sandry
+Sandovar
+Sandos
+Sandone
+Sandness
+Sandlan
+Sandison
+Sandersen
+Sandborg
+Sanchz
+Sanchec
+Sancen
+Sanasith
+Samway
+Samuell
+Sampselle
+Sampieri
+Sampair
+Samoyoa
+Samowitz
+Sammut
+Samiec
+Samick
+Samele
+Sambucetti
+Samara
+Samantha
+Samanlego
+Salverson
+Salvature
+Saluto
+Saluja
+Saltourides
+Saltmarsh
+Salta
+Salsberg
+Saloum
+Salos
+Saloom
+Sallings
+Sallies
+Sallah
+Salisberry
+Salimas
+Salfelder
+Salesses
+Salen
+Saleado
+Saldvir
+Saldi
+Saldeen
+Salceda
+Salazan
+Salaza
+Salay
+Salandy
+Sakshaug
+Sakovitch
+Sakkinen
+Sakkas
+Sakiestewa
+Sakic
+Sakakeeny
+Saison
+Saisa
+Saintfleur
+Saide
+Saicedo
+Sahsman
+Sahli
+Sahler
+Sahlberg
+Sahagian
+Saggione
+Sages
+Sagendorf
+Safron
+Safar
+Saetteurn
+Saenphimmacha
+Sadhu
+Sadhra
+Saden
+Sadee
+Saddat
+Sackos
+Sachleben
+Saches
+Sachar
+Saccucci
+Sacane
+Sablone
+Sablock
+Sablea
+Sabiston
+Sabini
+Sabi
+Sabha
+Sabellico
+Sabaj
+Saadd
+Ryun
+Rysavy
+Rysanek
+Rylowicz
+Ryll
+Ryken
+Rygiewicz
+Rydalch
+Rychlicki
+Rybowiak
+Ryal
+Ruzycki
+Ruyz
+Ruwet
+Rutley
+Ruthenberg
+Ruszala
+Rusteika
+Rusteberg
+Russotto
+Russotti
+Russman
+Russek
+Russe
+Rusley
+Rusich
+Rushworth
+Rushman
+Rushforth
+Ruscitti
+Ruscio
+Ruschmann
+Ruschel
+Rusak
+Rupertus
+Ruoho
+Runzler
+Runyons
+Runswick
+Runfola
+Rumney
+Rummler
+Rumford
+Rumburd
+Rumbold
+Ruman
+Rulnick
+Rujawitz
+Ruhstorfer
+Ruhmann
+Ruhling
+Ruhlin
+Ruggiere
+Ruggero
+Rugga
+Rugama
+Ruffolo
+Ruether
+Ruesswick
+Ruell
+Rudnitski
+Rudnicky
+Rudish
+Rudicil
+Rudes
+Rudeen
+Rubow
+Rubloff
+Rubison
+Rubinow
+Ruberte
+Rubenacker
+Rubarts
+Ruballos
+Rubal
+Rozgonyi
+Rozga
+Rozenberg
+Rozas
+Rozance
+Roytek
+Rowsell
+Rowray
+Rowold
+Rowntree
+Rowlins
+Rowling
+Rowback
+Rovelto
+Rovella
+Rovack
+Rouzzo
+Rout
+Roussos
+Rounkles
+Roundabush
+Rouisse
+Rougier
+Rouff
+Roudybush
+Roucoulet
+Roubekas
+Rotstein
+Rothmann
+Rothhaupt
+Rothfus
+Rothenburger
+Rothbauer
+Rothacher
+Rotering
+Roszales
+Rossnagel
+Rossingnol
+Rossing
+Rosselle
+Roskovensky
+Roskop
+Rositano
+Rosine
+Rosich
+Rosettie
+Rosentrance
+Rosenthall
+Rosenkoetter
+Rosenheim
+Rosenbarger
+Rosekrans
+Rosebure
+Roseboom
+Roscow
+Roscorla
+Rosbozom
+Rosavio
+Rosacker
+Ropiski
+Ronzoni
+Rons
+Rondell
+Ronde
+Roncskevitz
+Romulus
+Rompf
+Romjue
+Romenesko
+Rombult
+Rombardo
+Romaniak
+Romandia
+Romanchuk
+Romag
+Rolseth
+Rollind
+Rollend
+Rolfsen
+Rolff
+Rolek
+Rokusek
+Rohs
+Rohowetz
+Rohlack
+Rohla
+Rogugbakaa
+Roguemore
+Rogosky
+Roginson
+Roggero
+Roggensack
+Roggenbaum
+Roggeman
+Roever
+Roetzler
+Roettgen
+Roessing
+Roerish
+Roemhild
+Roehling
+Roede
+Roeber
+Rodriuez
+Rodrigeuz
+Rodnguez
+Rodis
+Rodinson
+Rodine
+Rodemoyer
+Rodeigues
+Rodea
+Roddick
+Rodar
+Rodamis
+Rodal
+Rockymore
+Rockelman
+Rockafellow
+Rocho
+Rochlin
+Rochenstire
+Rocasah
+Roblow
+Roblodowski
+Robinzine
+Robinsons
+Robinso
+Robinault
+Robilotto
+Robichard
+Robeza
+Robertos
+Roberrtson
+Robblee
+Robante
+Roats
+Roatch
+Roaoo
+Roanhorse
+Roal
+Roacho
+Rizas
+Rivord
+Riveroll
+Riverman
+Rivel
+Ritzke
+Ritzie
+Ritums
+Ritson
+Ritchlin
+Ritari
+Ristaino
+Rissell
+Rissanen
+Risler
+Riskalla
+Risius
+Rishell
+Risha
+Risewick
+Risden
+Rische
+Riscen
+Risbeck
+Riquelme
+Ripoll
+Rioz
+Riofrio
+Riobe
+Rinnert
+Rinkus
+Rininger
+Ringland
+Ringhouse
+Ringelspaugh
+Rinebold
+Rindler
+Rinderle
+Rimm
+Rillera
+Riise
+Riippi
+Rightnour
+Rightley
+Riggings
+Rigger
+Riffee
+Rifenbery
+Riexinger
+Riesland
+Rieske
+Riesinger
+Rieley
+Riekert
+Rief
+Riedlinger
+Ridgnal
+Ridgle
+Ridgill
+Ridep
+Ridel
+Riddleberger
+Ridders
+Riculfy
+Rickford
+Richters
+Richmann
+Richlin
+Richiusa
+Richerds
+Richan
+Ricenberg
+Ricaud
+Ricardi
+Ribsamen
+Ribron
+Ribiero
+Ribero
+Ribbink
+Rhump
+Rhum
+Rhorer
+Rhoe
+Rhoan
+Rhoad
+Rhinerson
+Rhen
+Reznicek
+Reyner
+Reyne
+Reynaldo
+Reyelts
+Rewerts
+Rewakowski
+Revira
+Revils
+Revering
+Revera
+Revelli
+Revay
+Reuteler
+Reust
+Reuschel
+Reudink
+Retzloff
+Rethmeier
+Retek
+Retchless
+Retamar
+Ressel
+Respicio
+Respes
+Respers
+Resos
+Resetar
+Resenz
+Resecker
+Res
+Rerucha
+Requarth
+Reprogle
+Repoff
+Replin
+Repetowski
+Repasky
+Reola
+Renzoni
+Renzo
+Renyer
+Rentoulis
+Rentie
+Renouf
+Renosky
+Renigar
+Renert
+Rendler
+Rend
+Remondet
+Remis
+Remian
+Remele
+Remeder
+Rellama
+Rekus
+Rekemeyer
+Reives
+Reitter
+Reistetter
+Reinsvold
+Reinsfelder
+Reinowski
+Reinier
+Reing
+Reinen
+Reineccius
+Reindeau
+Reinbolt
+Reimnitz
+Reimmer
+Reihl
+Reihing
+Reigleman
+Reighley
+Reidherd
+Reidhaar
+Reichow
+Reibman
+Reial
+Rehse
+Rehmert
+Rehlander
+Reher
+Rehbock
+Regulski
+Regueira
+Regn
+Reginaldo
+Regelman
+Regar
+Refsal
+Refazo
+Reemer
+Reefer
+Redlon
+Redkey
+Redinbo
+Rediker
+Redig
+Redemer
+Redcross
+Redal
+Recuparo
+Recksiek
+Reckers
+Recidivi
+Rechichi
+Reburn
+Rebold
+Rebik
+Rebar
+Reavish
+Reaver
+Reavely
+Reash
+Reaollano
+Reagey
+Readinger
+Readdy
+Razon
+Rayyan
+Rayshell
+Rayow
+Rayome
+Rayhel
+Raychard
+Rayam
+Rawi
+Rawhouser
+Rawat
+Ravizee
+Raviele
+Ravago
+Rautenstrauch
+Raulino
+Raul
+Rauhecker
+Rauhe
+Raught
+Rauco
+Raucci
+Ratzloff
+Rattu
+Rattell
+Rattanasinh
+Ratsep
+Ratkovich
+Rathrock
+Rathel
+Rathai
+Ratana
+Rasual
+Rastetter
+Rastegar
+Rasset
+Raspotnik
+Raspa
+Rasool
+Rasole
+Rasley
+Raskey
+Rasico
+Rasavong
+Ras
+Rarogal
+Rarden
+Raptis
+Rappl
+Rapkowicz
+Rapisura
+Rapanot
+Rapalo
+Rapacki
+Ranweiler
+Ransonet
+Ransler
+Ranni
+Ranmar
+Ranks
+Ranildi
+Randgaard
+Randahl
+Ranch
+Ranaudo
+Ranah
+Ramsy
+Ramsour
+Ramshur
+Ramsby
+Ramrirez
+Rampy
+Rampulla
+Rampadarat
+Rampa
+Ramonez
+Ramler
+Ramlall
+Ramjhon
+Ramjan
+Ramirel
+Rametta
+Ramelli
+Ramelize
+Ramelb
+Ramdeo
+Ramcharran
+Ramaudar
+Ramal
+Ramagano
+Ramach
+Rakyta
+Rakus
+Rakestrow
+Rakers
+Rajk
+Rajas
+Rajaphoumy
+Raisley
+Raisler
+Raisin
+Rais
+Railes
+Raike
+Raigosa
+Rahoche
+Rahmes
+Rahib
+Rahaman
+Ragus
+Ragula
+Raguay
+Raglow
+Rafus
+Rafey
+Rafel
+Rafala
+Raethke
+Raemer
+Raef
+Raeder
+Radziwon
+Radwick
+Radwanski
+Radoslovich
+Radon
+Radmall
+Radlinski
+Radie
+Raderstorf
+Radej
+Raddle
+Raczak
+Racko
+Raciti
+Racioppo
+Racer
+Rabuse
+Rabsatt
+Rabjohn
+Rabito
+Rabey
+Rabeneck
+Rabehl
+Rabeck
+Rabbe
+Rabal
+Quivoz
+Quiver
+Quituqua
+Quitugua
+Quittner
+Quitter
+Quitero
+Quitedo
+Quirke
+Quiram
+Quiralte
+Quintard
+Quintania
+Quinnan
+Quinlivan
+Quilter
+Quillman
+Quillan
+Quilindrino
+Quiel
+Quidas
+Quicho
+Quibodeaux
+Quezergue
+Quezad
+Quettant
+Queros
+Querio
+Quercioli
+Quenzel
+Quencer
+Queller
+Quebral
+Quatrevingt
+Quashnock
+Quasdorf
+Quartuccio
+Quartiero
+Quartieri
+Quartaro
+Quarrell
+Quanstrum
+Quammen
+Qualheim
+Quagliato
+Quadnau
+Qua
+Qasba
+Qare
+Qadeer
+Pywell
+Pysher
+Pyros
+Pyfrom
+Pyfer
+Pyette
+Pychardo
+Puzon
+Putzer
+Putton
+Putcha
+Puskarich
+Push
+Purkhiser
+Purfeerst
+Puraty
+Puotinen
+Puntillo
+Punihaole
+Pundsack
+Puna
+Pulwer
+Pullus
+Pullara
+Puita
+Puhrman
+Puhr
+Puhl
+Puffenberger
+Puerto
+Puent
+Pudenz
+Pucket
+Pucker
+Public
+Ptaschinski
+Psuty
+Psuik
+Psilovikos
+Przybyl
+Przeniczny
+Prye
+Prybylski
+Prukop
+Pruessner
+Provosty
+Provorse
+Provins
+Provino
+Provenzo
+Provent
+Protich
+Protas
+Pross
+Prosienski
+Prosenick
+Proscia
+Prosak
+Propheter
+Promisco
+Promer
+Prokup
+Prokos
+Progl
+Profeta
+Profera
+Profancik
+Procsal
+Prociuk
+Prochak
+Proch
+Procaccino
+Prizio
+Privado
+Pritzker
+Pritzel
+Pritcher
+Pritchell
+Prisoc
+Priolean
+Prinn
+Prindiville
+Princevalle
+Primos
+Prima
+Prigg
+Priego
+Priegnitz
+Prible
+Pribish
+Pribbenow
+Prevot
+Prevet
+Pretzer
+Pretzel
+Prety
+Presume
+Prestley
+Prestipino
+Presnal
+Preslipsky
+Presiado
+Prendes
+Prejsnar
+Preist
+Preissner
+Preisner
+Preheim
+Prefontaine
+Predom
+Precissi
+Prechtel
+Precht
+Prause
+Pratten
+Prately
+Prante
+Prang
+Pramuk
+Praley
+Prakoth
+Prach
+Pozar
+Poynton
+Powskey
+Powsey
+Powlen
+Powells
+Pourvase
+Pourner
+Pourier
+Pourchot
+Pouncil
+Poulisse
+Poulet
+Pouk
+Pouche
+Potulski
+Pottkotter
+Pottichen
+Potteiger
+Potsander
+Pothoven
+Potanovic
+Potaczala
+Posusta
+Posto
+Postles
+Postiglione
+Postemski
+Possinger
+Possick
+Possehl
+Pospicil
+Poskitt
+Poska
+Posis
+Portnoff
+Portello
+Porris
+Porres
+Porep
+Porell
+Porat
+Popularis
+Poppo
+Popadiuk
+Pooyouma
+Pooschke
+Poort
+Poolheco
+Ponsler
+Poniatowski
+Pomykala
+Pompi
+Pomilla
+Pomiecko
+Pomfret
+Polzer
+Polvino
+Poltrock
+Polton
+Polter
+Polski
+Poloskey
+Pollot
+Pollnow
+Polivick
+Polisoto
+Polintan
+Poliks
+Polikoff
+Policicchio
+Policastri
+Policare
+Poletski
+Polee
+Poledore
+Polacco
+Pokrzywa
+Pokallas
+Pointe
+Poinelli
+Pohorilla
+Pohlson
+Pogozelski
+Pogorelc
+Poellinetz
+Podwoski
+Podeszwa
+Pod
+Pocklington
+Pociengel
+Pochatko
+Pocekay
+Pocai
+Poague
+Pniewski
+Plutt
+Plumbar
+Pluma
+Plotzker
+Plotrowski
+Ploskunak
+Ploennigs
+Plimpton
+Plienis
+Plewinski
+Plett
+Pleskac
+Pleshe
+Plesant
+Pleppo
+Plegge
+Playl
+Plavnik
+Plateroti
+Plateros
+Plastow
+Plassmeyer
+Plassman
+Planer
+Plance
+Planagan
+Plan
+Plamondin
+Plainy
+Plackett
+Placino
+Plachecki
+Placeres
+Plaas
+Pjetrovic
+Pizzulo
+Pizzini
+Pizzico
+Pivec
+Pitpitan
+Pitorak
+Pitocco
+Pitka
+Pitch
+Pitcairn
+Pitarresi
+Piszczek
+Pistelli
+Piskel
+Pisicchio
+Piserchio
+Piscitello
+Pirrotta
+Pirrello
+Pirre
+Pirozhkov
+Pirollo
+Pirieda
+Pipper
+Pipia
+Pioske
+Piombino
+Pinzino
+Pintello
+Pinsonneault
+Pinsoneault
+Pinn
+Pinkenburg
+Pinke
+Pindell
+Pinchock
+Pince
+Pimple
+Pim
+Piluso
+Pillon
+Pillarella
+Pillado
+Pilkey
+Pilette
+Pilchowski
+Piirto
+Pihlaja
+Piggie
+Piganelli
+Piety
+Pietrowicz
+Pietrok
+Pietrini
+Piesco
+Piertraccini
+Piersiak
+Pierrot
+Pierdon
+Pierannunzio
+Pientka
+Pielow
+Piela
+Piek
+Piegaro
+Piefer
+Piecuch
+Pidro
+Picotte
+Pickman
+Picketts
+Picketpin
+Pickerell
+Pickenpaugh
+Pichoff
+Picher
+Piccuillo
+Piccirilli
+Piccinone
+Piccinich
+Piccillo
+Picchetti
+Piatz
+Piao
+Piacitelli
+Piacenza
+Phyfe
+Phurrough
+Phuong
+Phuma
+Phuaphes
+Phramany
+Phoubandith
+Phommajack
+Phom
+Pho
+Phimsoutham
+Phimpradapsy
+Philmore
+Phillies
+Philliber
+Philio
+Phildor
+Philabaum
+Phi
+Phetsanghane
+Phetphongsy
+Phelp
+Phaymany
+Pharmer
+Pharao
+Phanthavongsa
+Pfrommer
+Pfoutz
+Pforr
+Pfnister
+Pflugradt
+Pflugrad
+Pfleuger
+Pfingsten
+Pfifer
+Pfeiffenberge
+Pfefferkorn
+Pfanstiel
+Pfander
+Pfalmer
+Pfaffinger
+Pezley
+Pezina
+Pezez
+Peyser
+Pevahouse
+Petula
+Petton
+Pettipas
+Pettijohn
+Pettigrove
+Pettay
+Petrouits
+Petropulos
+Petronzio
+Petronella
+Petrilli
+Petriccione
+Petric
+Petrecca
+Petralia
+Petr
+Petka
+Petigny
+Petesic
+Petersik
+Petek
+Petanick
+Petalcu
+Peszynski
+Pessolano
+Pesses
+Pesicka
+Peschong
+Pesarchick
+Pesantes
+Perza
+Pertea
+Persyn
+Persten
+Persch
+Perrota
+Perrot
+Perriott
+Perring
+Perrilloux
+Perrette
+Perrelli
+Perrell
+Pernod
+Pernin
+Perniciaro
+Pernesky
+Permann
+Perlson
+Perkiss
+Perina
+Perie
+Perencevich
+Peredz
+Percey
+Peraha
+Peplau
+Pepka
+Pepion
+Penzien
+Penzel
+Penya
+Penwarden
+Penticoff
+Pensky
+Pensick
+Pensa
+Pennelle
+Penird
+Penhallurick
+Penha
+Pengra
+Penderel
+Pendegraft
+Pencak
+Pemelton
+Peluse
+Pelnar
+Pellom
+Pellitteri
+Pelligrino
+Pellietier
+Pellicone
+Pelletiu
+Pellet
+Pellam
+Peleg
+Pekas
+Pekara
+Pehowich
+Peha
+Pegeron
+Peffly
+Pefferkorn
+Peetoom
+Peerzada
+Peecha
+Peduzzi
+Pedralba
+Pedez
+Pedeare
+Pecinousky
+Pechaira
+Pecatoste
+Pecarina
+Pecararo
+Pearyer
+Peacy
+Peachay
+Payseur
+Payor
+Payna
+Payant
+Payamps
+Pax
+Pawluch
+Pavliska
+Pavis
+Pavelski
+Pavella
+Pav
+Pauza
+Pausch
+Paulshock
+Paulseth
+Paulmino
+Paulic
+Paulauskis
+Paulauskas
+Paulas
+Pauker
+Paugsch
+Patzner
+Patzke
+Patwell
+Patuel
+Pattyre
+Pattinson
+Pattengale
+Patriquin
+Patrin
+Patrias
+Patria
+Patolot
+Patik
+Paterniti
+Patellis
+Patches
+Patcher
+Patanella
+Pataki
+Patajo
+Pasvizaca
+Pastures
+Pasto
+Pastian
+Passerino
+Passer
+Paskow
+Pasket
+Pasinski
+Pasho
+Pashea
+Pashal
+Pascorell
+Pascoal
+Pascanik
+Pascall
+Pasaya
+Pasana
+Paruta
+Party
+Partman
+Partipilo
+Partenope
+Partelow
+Part
+Parsygnat
+Parsh
+Parsells
+Parrotta
+Parron
+Parrington
+Parrin
+Parriera
+Parreno
+Parquette
+Parpan
+Parone
+Parnin
+Parms
+Parmantier
+Parkos
+Parkhouse
+Parizek
+Paripovich
+Parinas
+Parihar
+Parhan
+Pargman
+Pardoe
+Parayuelos
+Paravano
+Paratore
+Parara
+Papranec
+Pappajohn
+Paponetti
+Papitto
+Papike
+Papiernik
+Papciak
+Papantonio
+Papanikolas
+Papania
+Papan
+Papale
+Pap
+Paongo
+Paola
+Panzica
+Panzella
+Panyko
+Panuccio
+Pantosa
+Pantoliano
+Pantelakis
+Panrell
+Panowicz
+Panora
+Pankiw
+Pankake
+Panitz
+Panila
+Panias
+Paneque
+Panela
+Paneczko
+Pandola
+Panahon
+Panah
+Panagoulias
+Panagis
+Paluszynski
+Paluk
+Paluck
+Palu
+Paloukos
+Palombit
+Palmios
+Palley
+Pallant
+Pallansch
+Pallafor
+Palisbo
+Palchetti
+Palazola
+Palas
+Palacois
+Pakonen
+Pajerski
+Paillant
+Pahk
+Pagni
+Pagnello
+Paglio
+Paga
+Pafel
+Padol
+Padgette
+Padeken
+Paddio
+Paddilla
+Paddack
+Padavich
+Pacquin
+Packineau
+Pacior
+Pacholec
+Pachlin
+Pachla
+Pach
+Pacenta
+Pacek
+Pacapac
+Pacana
+Paben
+Paarmann
+Paalan
+Ozer
+Ozane
+Ozaine
+Ozaeta
+Oz
+Oyston
+Oyellette
+Oxton
+Oxnam
+Oxenrider
+Oxborough
+Owers
+Ow
+Ovit
+Ovesen
+Overstrom
+Overshiner
+Overmire
+Overley
+Overkamp
+Overdick
+Overbough
+Ovdenk
+Ovadilla
+Ouye
+Outzen
+Ousdahl
+Oury
+Ourth
+Ounsy
+Ouellete
+Oudker
+Otutaha
+Otuafi
+Ottrix
+Ottogary
+Ottino
+Ottilige
+Ottenwess
+Otiz
+Othoudt
+Otex
+Otega
+Osvaldo
+Ostwald
+Ostrzyeki
+Ostrum
+Ostroot
+Osterhaut
+Ostendorff
+Ostenberg
+Ostasiewicz
+Osswald
+Ossola
+Osowicz
+Osorno
+Osollo
+Osol
+Osnoe
+Osmus
+Osmanski
+Osias
+Oshman
+Osentowski
+Osden
+Osche
+Osbeck
+Orttenburger
+Ortolf
+Orto
+Ortga
+Orrego
+Orpin
+Orozeo
+Orochena
+Orobona
+Oroark
+Ornelos
+Ornedo
+Orne
+Orm
+Orlove
+Orlosky
+Orlof
+Orlinsky
+Orlinski
+Orlin
+Orizabal
+Oriti
+Orion
+Origer
+Orie
+Orhenkowski
+Orford
+Orff
+Oreskovich
+Orellama
+Oreily
+Orehek
+Oreb
+Ordazzo
+Ordahl
+Orcholski
+Orce
+Oras
+Opula
+Opstein
+Oppliger
+Oppegard
+Opichka
+Opher
+Opet
+Opalicki
+Opaka
+Ooton
+Onyeanus
+Onwunli
+Onukogu
+Onisick
+Onifade
+Oneale
+Ondik
+Ondic
+Ondersma
+Omullan
+Omoto
+Omo
+Omlin
+Omli
+Omersa
+Olverson
+Olveira
+Olvedo
+Olowe
+Olona
+Olnes
+Olloqui
+Olliver
+Ollhoff
+Ollendick
+Olkowski
+Olivid
+Olivers
+Oliveres
+Olivarra
+Olinghouse
+Oligee
+Olgvin
+Olfers
+Olewinski
+Olewine
+Oleveda
+Oleskiewicz
+Olejarski
+Olecki
+Olde
+Olckhart
+Olbrish
+Olay
+Olarte
+Okwuona
+Okuley
+Okula
+Okorududu
+Okoren
+Okoli
+Okihara
+Okerson
+Oken
+Ojard
+Ojanen
+Oines
+Oilvares
+Oieda
+Ohrnstein
+Ohren
+Ohmit
+Ohmie
+Ohlmacher
+Ohlenbusch
+Ohlen
+Ohaver
+Oharroll
+Ogwynn
+Ogunyemi
+Ogram
+Ogilive
+Ogen
+Ogbonnaya
+Ogasawara
+Ogans
+Ogami
+Oflahrity
+Offret
+Oen
+Oeler
+Oehrlein
+Oehrle
+Oehmke
+Oehmig
+Oeftger
+Oeder
+Odougherty
+Odorizzi
+Odomes
+Odin
+Odien
+Odhner
+Odess
+Odenheimer
+Ocus
+Ochsenbein
+Ochinang
+Ochiai
+Ochalek
+Occhino
+Ocacio
+Obnegon
+Oblow
+Oblinger
+Obiano
+Obery
+Oberson
+Oberpriller
+Obermuller
+Obermoeller
+Oberholzer
+Oberhaus
+Oberdier
+Oberdick
+Oaxaca
+Oar
+Nysether
+Nykiel
+Nygaro
+Nycum
+Nyahay
+Nwankwo
+Nwakanma
+Nwadiora
+Nwabeke
+Nuzenski
+Nusz
+Nunnelee
+Nunmaker
+Nuniz
+Nunery
+Nulisch
+Nuetzman
+Nuessle
+Nuesca
+Nuckoles
+Nuccitelli
+Nucci
+Nozum
+Nozick
+Nowzari
+Nowosadko
+Nowley
+Nowitzke
+Novitsky
+Novitski
+Novitske
+Novikoff
+Novida
+Novetsky
+Novelly
+Novellino
+Novara
+Nouth
+Noullet
+Noud
+Notwick
+Notowitz
+Notley
+Notis
+Nothem
+Nothacker
+Nostro
+Noseff
+Norwell
+Northwood
+Northcut
+Norstrud
+Norseth
+Norse
+Norsaganay
+Norko
+Norkaitis
+Noriego
+Norg
+Noreiga
+Nordwall
+Nordsiek
+Nordlinger
+Nordick
+Nordenstrom
+Norbo
+Noorigian
+Noordam
+Nonu
+Nones
+Noneman
+Nondorf
+Noltensmeier
+Nollette
+Nolfe
+Nolazco
+Nokken
+Noke
+Noiseux
+Noia
+Nohe
+Nogueda
+Noguchi
+Nogoda
+Noggles
+Noggler
+Noftsier
+Noey
+Noerenberg
+Noegel
+Nodurft
+Nodarse
+Nockai
+Nobregas
+Nobis
+Nkuku
+Nkomo
+Njango
+Niziol
+Nixion
+Nixa
+Nivar
+Nivala
+Nitzschke
+Nitzsche
+Nitzkowski
+Nitcher
+Niswender
+Nisley
+Nishimori
+Nirmaier
+Nipps
+Nipple
+Ninke
+Nini
+Ninh
+Nimrod
+Nimox
+Nimick
+Nila
+Niksich
+Nikodem
+Nikocevic
+Nikaido
+Nightlinger
+Niggemann
+Nietfeldt
+Niess
+Niesent
+Niesborella
+Nierer
+Niemitzio
+Niemiel
+Niemants
+Niedzwiedzki
+Niedzwiedz
+Niedens
+Niedbalec
+Niebaum
+Nicoson
+Nicoli
+Nicolaus
+Nickoley
+Nicklos
+Nicklien
+Nickenberry
+Nickas
+Nicholason
+Nichell
+Nichalson
+Nicewonger
+Niau
+Nian
+Nham
+Nguyan
+Ngin
+Nezich
+Nezat
+Neyaci
+Newstead
+Newness
+Newhook
+Newes
+Newens
+Newbell
+Newball
+Nevinger
+Nevilles
+Nevil
+Never
+Nevarrez
+Neuse
+Neundorfer
+Neuenswander
+Neudeck
+Neubig
+Neubaum
+Neubacher
+Nettleingham
+Netrosio
+Netolicky
+Netley
+Nesti
+Nessmith
+Neslusan
+Nesline
+Nesland
+Nesin
+Nerlich
+Nepa
+Neonakis
+Nenni
+Nemzin
+Nemunaitis
+Nemets
+Nemard
+Nemani
+Nelmes
+Nellums
+Nellenback
+Nelisse
+Nejaime
+Neja
+Neither
+Neiswoger
+Neiper
+Neild
+Neidiger
+Nehrt
+Nehme
+Neglio
+Negbenebor
+Needy
+Nedman
+Nedina
+Nederostek
+Nedelman
+Neddo
+Nedbalek
+Nebred
+Neblock
+Nebesnik
+Nebarez
+Neall
+Nealious
+Nealer
+Neahr
+Ncneal
+Nazzise
+Nazzal
+Nazir
+Nazelrod
+Naz
+Naysmith
+Nayman
+Nawwar
+Nawda
+Naveed
+Navarrate
+Navaretta
+Navappo
+Navanjo
+Natwick
+Nattiah
+Natsis
+Nati
+Nathans
+Natewa
+Natani
+Natalello
+Nasti
+Nassie
+Nasr
+Nasers
+Nasalroad
+Narr
+Nargi
+Nardy
+Napieralski
+Nanthanong
+Nantanapibul
+Nanna
+Nanik
+Nanasy
+Nanas
+Namur
+Namihira
+Namaka
+Nalty
+Nalbach
+Naki
+Nakatsu
+Nakamori
+Najarian
+Nailer
+Naifeh
+Naidu
+Nahrwold
+Nahl
+Nahari
+Nagode
+Nagindas
+Nagengast
+Nagelhout
+Nagase
+Naftzinger
+Naftali
+Naeher
+Nadoff
+Naderi
+Nadelbach
+Naddeo
+Nacy
+Nacisse
+Nacion
+Nachtrieb
+Nachmias
+Nachazel
+Nacar
+Naborg
+Nabity
+Nabhan
+Mytych
+Myslinski
+Myslin
+Mysak
+Myrtle
+Myrman
+Myrck
+Myntti
+Mynnerlyn
+Mylott
+Myking
+Myes
+Mycroft
+Mway
+Muzyka
+Muzacz
+Muyskens
+Muysenberg
+Mutone
+Mutner
+Mutherspaw
+Muthart
+Muthana
+Mutart
+Musty
+Muston
+Mussmann
+Musshorn
+Musse
+Muss
+Musquiz
+Musolf
+Muskthel
+Muska
+Musinski
+Musigdilok
+Muschick
+Muschett
+Musch
+Murwin
+Murty
+Mursko
+Murnock
+Mure
+Murasso
+Muraro
+Muran
+Murallies
+Muraco
+Munyer
+Munshi
+Munning
+Munl
+Munir
+Muninger
+Munhall
+Muney
+Munet
+Mundziak
+Mundschau
+Mundhenk
+Munderville
+Muncil
+Munchmeyer
+Munaz
+Muna
+Mulzer
+Mulvahill
+Mulryan
+Mulroney
+Mulready
+Mulneix
+Mullowney
+Mullner
+Mullison
+Mullany
+Mulich
+Mula
+Muhtaseb
+Muhlenkamp
+Muhlbach
+Muggley
+Mueske
+Muenkel
+Muell
+Muehleisen
+Mudrick
+Muddaththir
+Muczynski
+Mucklow
+Muckley
+Muckelvaney
+Muchortow
+Mthimunye
+Mrazik
+Mozzone
+Mozo
+Mozley
+Mozie
+Mozgala
+Mozelak
+Moyerman
+Mowder
+Mowan
+Movlin
+Mouzas
+Mourino
+Moulhem
+Mottillo
+Motteshard
+Mottershead
+Motamed
+Mosz
+Mostoller
+Mostiller
+Mostero
+Mostella
+Mosson
+Mossing
+Mossien
+Mossel
+Mosmeyer
+Moskau
+Moshos
+Mosho
+Moscovic
+Moscaritolo
+Moscariello
+Moscardelli
+Morosow
+Morono
+Morneault
+Morna
+Morn
+Morkve
+Moriwaki
+Morise
+Moriera
+Moricle
+Moribayed
+Morgret
+Morgner
+Morgas
+Morgans
+Morgandi
+Morfee
+Morelen
+Moreida
+Moreci
+Moreb
+Mordino
+Mordini
+Mordehay
+Morda
+Mootz
+Mootispaw
+Moosbrugger
+Moosa
+Moonsommy
+Moonshower
+Moodispaugh
+Mooberry
+Monz
+Montuoro
+Montrella
+Montijano
+Montgonery
+Montelle
+Montell
+Montcalm
+Montalgo
+Monske
+Monrroy
+Monrow
+Monnot
+Moniak
+Mongue
+Mongolo
+Mongiovi
+Monfore
+Mondoux
+Mondone
+Mondell
+Mondaine
+Moncrieffe
+Moncrieff
+Moncier
+Monasterio
+Monarque
+Monaham
+Monagle
+Momper
+Momeni
+Moltrie
+Molone
+Molly
+Mollohan
+Molliere
+Mollere
+Molleker
+Mollberg
+Molinini
+Moling
+Molineaux
+Molett
+Moldan
+Molavi
+Molaison
+Mokriski
+Mokiao
+Mojzisik
+Mojardin
+Moisey
+Mohorovich
+Mohinani
+Mohaupt
+Mohabeer
+Mogollon
+Moghadam
+Mofle
+Mofford
+Moevao
+Moelter
+Moede
+Modrak
+Moddejonge
+Mockler
+Mocha
+Mobilio
+Mlenar
+Mizzi
+Mizner
+Mizee
+Miyasaka
+Miyao
+Mixdorf
+Mitter
+Mittchell
+Mittag
+Mithani
+Mitchler
+Misove
+Mismit
+Misluk
+Miskovich
+Mishou
+Miserendino
+Misek
+Miscoe
+Mirmow
+Mirman
+Mirkovich
+Mirao
+Miran
+Miquelon
+Minucci
+Mintreas
+Mintos
+Mintor
+Minotti
+Minock
+Minnatee
+Miniuk
+Minissale
+Minihan
+Minicozzi
+Mini
+Minford
+Minette
+Minery
+Minehan
+Mineconzo
+Mindingall
+Minchella
+Minarcik
+Minacci
+Mimaki
+Milz
+Milwee
+Miltz
+Milsaps
+Milosevich
+Millstead
+Millott
+Millora
+Millian
+Millhiser
+Millerr
+Millbrand
+Millbern
+Millberg
+Milkent
+Milius
+Milite
+Milelr
+Mildred
+Milderberger
+Mildenstein
+Milbrodt
+Milare
+Mikulec
+Mikovec
+Mikota
+Mikolon
+Mikhaiel
+Mikez
+Miker
+Mikasa
+Mihovk
+Mihor
+Mihaliak
+Mihalco
+Mihalak
+Miggo
+Miessler
+Miernik
+Miernicki
+Miene
+Mieloszyk
+Mielkie
+Mielczarek
+Mielcarz
+Miehe
+Midget
+Middough
+Middents
+Microni
+Mickulskis
+Micks
+Mickonis
+Mickenheim
+Michello
+Michealson
+Michavd
+Michalczik
+Mezzinni
+Mezzanotte
+Meysembourg
+Meyerowitz
+Meyerott
+Meyerman
+Meyerhoefer
+Mevis
+Mevers
+Meuler
+Meulemans
+Meua
+Metzga
+Metzel
+Mettlen
+Mettille
+Metott
+Metos
+Metil
+Metia
+Metherell
+Metevelis
+Metenosky
+Meteer
+Metchikoff
+Mestler
+Mestanza
+Messman
+Messey
+Messervy
+Messel
+Messan
+Mesoloras
+Mesmer
+Mesiona
+Mesias
+Meshew
+Meshanko
+Meservy
+Mesecar
+Mesdaq
+Merzig
+Mervine
+Mertine
+Merrills
+Merren
+Merlette
+Merles
+Merlain
+Merl
+Merksamer
+Merithew
+Merisier
+Mering
+Merilos
+Merical
+Merhar
+Merette
+Mereno
+Merdian
+Merceir
+Mercando
+Merante
+Merana
+Merales
+Menucci
+Mentkowski
+Mentgen
+Menso
+Mensen
+Menkin
+Menjes
+Menjares
+Menitz
+Menietto
+Menier
+Meneus
+Menefield
+Menees
+Mendrin
+Mendrala
+Mendler
+Mendiaz
+Mendesa
+Mencke
+Menchu
+Menches
+Menas
+Mems
+Memo
+Memmo
+Meltzner
+Melter
+Melstrom
+Melsheimer
+Melser
+Melodia
+Mellos
+Mellis
+Melliere
+Mellie
+Mellecker
+Mellage
+Mellady
+Melikyan
+Melford
+Meley
+Melencamp
+Meleen
+Melear
+Melchert
+Melaun
+Melaro
+Melady
+Mekonis
+Meisenburg
+Meireles
+Meinsen
+Meinershagen
+Meil
+Meihofer
+Mehrotra
+Mehlhaff
+Mehis
+Mehelich
+Mehdizadeh
+Mehdi
+Meharry
+Mehalko
+Megraw
+Megown
+Mego
+Megill
+Megia
+Meggison
+Meggett
+Meggerson
+Meetze
+Meeroff
+Meemken
+Meehleder
+Meeds
+Medure
+Medosch
+Medora
+Mednis
+Medling
+Medland
+Medious
+Medino
+Medin
+Medill
+Medieros
+Medi
+Medhus
+Medearis
+Medanich
+Medalion
+Meckel
+Meccia
+Mecardo
+Measheaw
+Measeck
+Mearing
+Meara
+Meakin
+Mcwilson
+Mcward
+Mcwalters
+Mcwade
+Mcvoy
+Mctush
+Mctiernan
+Mctarnaghan
+Mcswiggan
+Mcstay
+Mcritchie
+Mcrill
+Mcquiddy
+Mcqueeny
+Mcpharlane
+Mcphan
+Mcpartlin
+Mcnutty
+Mcnuh
+Mcnicoll
+Mcnicol
+Mcnevin
+Mcnespey
+Mcneme
+Mcnellie
+Mcnayr
+Mcmina
+Mcmenamy
+Mcmanigal
+Mcluckie
+Mclilly
+Mcleskey
+Mclearan
+Mclauchlen
+Mclatchy
+Mclaen
+Mckray
+Mckouen
+Mckoon
+Mckisson
+Mckinna
+Mckines
+Mckimmy
+Mckimley
+Mckewen
+Mckerrow
+Mckenzy
+Mckentie
+Mckemie
+Mckaskle
+Mckanic
+Mcintyde
+Mcinroy
+Mcinnish
+Mcilwaine
+Mciltrot
+Mchalffey
+Mcgurren
+Mcgurr
+Mcgunnis
+Mcgunnigle
+Mcgunagle
+Mcguinnes
+Mcguin
+Mcgrotha
+Mcgrogan
+Mcgraph
+Mcgoon
+Mcglothern
+Mcgloster
+Mcglohon
+Mcglockton
+Mcglawn
+Mcginnity
+Mcginister
+Mcgilberry
+Mcgiboney
+Mcghin
+Mcghaney
+Mcgeeney
+Mcgeady
+Mcgartland
+Mcgarraugh
+Mcgaffey
+Mcgafferty
+Mcgaffee
+Mcfeeley
+Mcfan
+Mceneny
+Mcelwine
+Mcelreavy
+Mcelpraug
+Mcelmeel
+Mceirath
+Mceady
+Mcdunn
+Mcdonnall
+Mcdewitt
+Mcdermett
+Mcdeavitt
+Mcdearmont
+Mccurine
+Mccunn
+Mccumbers
+Mccumbee
+Mccullors
+Mccullon
+Mccullogh
+Mccullock
+Mccuan
+Mccrate
+Mccra
+Mccoulskey
+Mccornack
+Mccormik
+Mccorkindale
+Mccorison
+Mcconnal
+Mccomack
+Mccole
+Mccoil
+Mccoard
+Mcclurken
+Mcclodden
+Mcclod
+Mcclimens
+Mccleveland
+Mcclenningham
+Mcclellon
+Mcclaugherty
+Mcclatcher
+Mcclarty
+Mcclamma
+Mcclaim
+Mcchain
+Mccelland
+Mccastle
+Mccarvill
+Mccarther
+Mccarr
+Mccarns
+Mccarn
+Mccard
+Mccandrew
+Mccandliss
+Mccalvin
+Mccalpin
+Mccalment
+Mccallun
+Mccallough
+Mccahan
+Mccaffree
+Mcbratney
+Mcaveney
+Mcausland
+Mcauly
+Mcarthun
+Mcanaw
+Mcall
+Mbamalu
+Mazzera
+Mazze
+Mazzawi
+Mazzaferro
+Mazzacano
+Mazuo
+Mazion
+Mazey
+Maywood
+Mayshack
+Mayrose
+Mayou
+Mayorca
+Mayoka
+Maynerich
+Maylone
+Mayhood
+Mayeshiba
+Maydew
+Maxi
+Maxell
+Mawhinney
+Mavropoulos
+Mavle
+Mavai
+Mautte
+Mauson
+Mausey
+Mauseth
+Mausbach
+Maurus
+Maurizio
+Maura
+Maupredi
+Maung
+Maultasch
+Mauleon
+Maud
+Matyi
+Matuszak
+Matushevsky
+Matusek
+Matuck
+Mattys
+Mattsey
+Mattione
+Mattias
+Matteis
+Matsu
+Matsoukas
+Matrey
+Matot
+Matlin
+Matkowsky
+Matise
+Mathwich
+Mathus
+Mathony
+Mathery
+Matherson
+Mathen
+Maten
+Matelich
+Matejek
+Matczak
+Matchen
+Matarrita
+Matakonis
+Mataka
+Matacale
+Masuyama
+Masure
+Masupha
+Masudi
+Masturzo
+Mastrocola
+Mastriano
+Mastrianni
+Mastrianna
+Mastrelli
+Massicotte
+Massetti
+Massella
+Massei
+Massee
+Massaquoi
+Masood
+Masom
+Maslowsky
+Masloski
+Maslonka
+Maski
+Maskaly
+Masiejczyk
+Masgalas
+Masero
+Masenten
+Masciantonio
+Masaya
+Masaracchia
+Marzocchi
+Marzili
+Marzigliano
+Marye
+Marusiak
+Marullo
+Marturano
+Martos
+Martorello
+Martineze
+Martillo
+Martignago
+Martiarena
+Marsters
+Marshalek
+Marsell
+Marsek
+Marseglia
+Marriot
+Marrion
+Marrington
+Marrietta
+Marrello
+Marreel
+Marrable
+Marquina
+Marque
+Marozzi
+Marovic
+Marotti
+Marose
+Marnett
+Marmolejos
+Markt
+Markson
+Marklund
+Markewich
+Marinoni
+Marinko
+Marinas
+Maril
+Mariello
+Marguardt
+Margreiter
+Margraf
+Margel
+Margaryan
+Margarita
+Margan
+Marevka
+Maresco
+Marero
+Marentez
+Maree
+Mardini
+Marcotrigiano
+Marcoguisepp
+Marcks
+Marcinka
+Marchizano
+Marchitto
+Marchiony
+Marchionese
+Marchesseault
+Marcheski
+Marchesano
+Marchall
+Marceaux
+Marbray
+Maratre
+Maratos
+Marashi
+Marasciulo
+Maras
+Marantz
+Marallo
+Maragni
+Maragh
+Marabella
+Maquis
+Maontesano
+Maobi
+Manzie
+Manzay
+Manvelito
+Manvel
+Manuell
+Mantik
+Mantele
+Mantegna
+Mansbridge
+Mansanares
+Manora
+Manolakis
+Manokey
+Mannine
+Mannheimer
+Mannebach
+Mannchen
+Manlito
+Mankoski
+Manivong
+Manheim
+Mangubat
+Manfra
+Manemann
+Manecke
+Mandry
+Mandler
+Mandi
+Mandap
+Mandahl
+Mancos
+Manciel
+Mancherian
+Manchel
+Manca
+Manby
+Manatt
+Manaker
+Mamone
+Mammano
+Malvern
+Malton
+Malsch
+Malovich
+Malouff
+Malory
+Maloff
+Malocha
+Malmanger
+Mallinger
+Mallinak
+Mallegni
+Mallat
+Malkoski
+Malinky
+Malinak
+Malichi
+Malgieri
+Maleszka
+Males
+Maleonado
+Malenke
+Malekan
+Malehorn
+Maleck
+Malcome
+Malay
+Malawy
+Malarkey
+Malanado
+Malama
+Malabey
+Makua
+Makhija
+Makel
+Makarem
+Majorga
+Majocka
+Majica
+Majic
+Majeau
+Maizes
+Mairot
+Maione
+Mainz
+Mainland
+Mainetti
+Mainero
+Maimone
+Maifeld
+Maiers
+Maiello
+Maidonado
+Maicus
+Mahung
+Mahula
+Mahrenholz
+Mahran
+Mahomly
+Mahin
+Mahe
+Mahall
+Mahal
+Magsby
+Magsayo
+Magrone
+Magraw
+Magrann
+Magpali
+Magouliotis
+Magorina
+Magobet
+Magnini
+Magnifico
+Magnie
+Magnett
+Maglioli
+Maggit
+Magg
+Magette
+Magdefrau
+Magdalena
+Magaziner
+Magathan
+Magalski
+Magaldi
+Magadan
+Mafua
+Maeno
+Maenaga
+Maedke
+Madziar
+Madre
+Madine
+Madin
+Madhavan
+Madge
+Madeja
+Maddoy
+Maddison
+Maddin
+Maddern
+Mad
+Macvicar
+Macurdy
+Macreno
+Macpartland
+Macoreno
+Macola
+Macnutt
+Macnevin
+Macmullan
+Maclain
+Mackstutis
+Macknair
+Macklem
+Mackillop
+Mackenthun
+Mackechnie
+Mackaman
+Macione
+Maciolek
+Maciarello
+Machover
+Machle
+Machi
+Machel
+Machak
+Macduffee
+Maccutcheon
+Macculloch
+Maccord
+Macconaghy
+Maccoll
+Macclellan
+Macclairty
+Maccini
+Macchiarella
+Maccheyne
+Maccarter
+Maccarino
+Maccarini
+Macandog
+Macanas
+Macalma
+Macabeo
+Maasen
+Maarx
+Lytell
+Lyson
+Lysher
+Lyngholm
+Lynchj
+Lynah
+Lyme
+Lyken
+Lyew
+Lydecker
+Lybert
+Lyberger
+Lybecker
+Lyau
+Lweis
+Luzi
+Luzell
+Luvianos
+Luvera
+Lutze
+Lutkus
+Luten
+Lusty
+Lustberg
+Lurye
+Lury
+Lurtz
+Luquette
+Lupiani
+Lupacchino
+Lunter
+Lunstrum
+Lungwitz
+Lungsford
+Lunemann
+Lunderman
+Lunch
+Luminati
+Lumbley
+Lumba
+Lumadue
+Lulas
+Lukow
+Lukianov
+Lukesh
+Lukander
+Luka
+Luing
+Luikart
+Lugabihl
+Lufborough
+Luette
+Luescher
+Lueschen
+Luersen
+Luensmann
+Luening
+Lueker
+Luedecke
+Lueckenbach
+Luebbering
+Ludovico
+Ludera
+Ludeker
+Ludecke
+Luczki
+Luco
+Luckinbill
+Lucis
+Lucik
+Lucie
+Lucic
+Luchterhand
+Luccous
+Lucash
+Luberger
+Lubbert
+Lubben
+Lubawy
+Lubahn
+Luangxay
+Luangrath
+Luangamath
+Luague
+Lozey
+Loyborg
+Loyack
+Loxton
+Loxtercamp
+Lownsbery
+Lowler
+Lowcks
+Lowa
+Lovstad
+Lovisone
+Lovfald
+Lovetinsky
+Lovet
+Lovero
+Loverdi
+Lovellette
+Loveberry
+Louwagie
+Lournes
+Louria
+Lourentzos
+Lourdes
+Louka
+Louil
+Loudermelt
+Louchen
+Loubier
+Lotto
+Lotridge
+Lothringer
+Lothridge
+Lota
+Lot
+Loszynski
+Lossius
+Losneck
+Loseth
+Losavio
+Losardo
+Losano
+Losado
+Losacco
+Losa
+Lorr
+Loron
+Lorincz
+Loria
+Loretz
+Lorentine
+Lordi
+Loraine
+Lopze
+Lopiccalo
+Lopey
+Loperfido
+Lope
+Lopata
+Lopas
+Loparco
+Loofbourrow
+Longwith
+Longhi
+Longenberger
+Longbine
+Longaker
+Longabaugh
+Lomonte
+Lomino
+Lominack
+Lomen
+Lombel
+Lombardino
+Lomago
+Loma
+Lokan
+Loiacona
+Lohry
+Lohrke
+Lohre
+Logoleo
+Loggens
+Logarbo
+Lofwall
+Lofty
+Lofts
+Lofthus
+Lofte
+Lofstrom
+Loforte
+Lofman
+Lofing
+Lofguist
+Loffier
+Loffelbein
+Loerwald
+Loeppky
+Loehrer
+Loehner
+Loecken
+Lockshaw
+Locknane
+Lockington
+Lockery
+Lockemer
+Lochrico
+Lobregat
+Lobley
+Lobello
+Lobell
+Lobalbo
+Lobach
+Llaneza
+Llanet
+Llams
+Livley
+Livinton
+Living
+Liversedge
+Livernois
+Livermon
+Liverance
+Liveoak
+Livecchi
+Livasy
+Liukkonen
+Litzenberger
+Litvak
+Littfin
+Litmanowicz
+Litchard
+Listi
+Listen
+Lisker
+Lisitano
+Lisena
+Lisbey
+Lipsie
+Lips
+Lippoldt
+Lippitt
+Lipper
+Lipoma
+Lipkovitch
+Lipira
+Lipan
+Linzan
+Linza
+Linsin
+Linsenmayer
+Linsdau
+Linnert
+Linman
+Linkon
+Lingner
+Lingley
+Lingerfelter
+Lingbeek
+Linero
+Lindorf
+Lindmeyer
+Lindinha
+Linderleaf
+Lindau
+Lindabury
+Linburg
+Linak
+Limmel
+Limle
+Limbert
+Limardi
+Lilyblade
+Lillehaug
+Likar
+Liiv
+Ligonis
+Ligler
+Lighthart
+Ligget
+Liftin
+Lifschitz
+Liewald
+Lievsay
+Lievens
+Lietzow
+Lierz
+Liegler
+Liedberg
+Lied
+Liebrecht
+Liebherr
+Lieberg
+Liebenthal
+Liebenow
+Liebeck
+Lidstone
+Lidie
+Lidge
+Lidder
+Licursi
+Licklider
+Lickfelt
+Lichota
+Lichenstein
+Liceaga
+Liccketto
+Libertini
+Libberton
+Leyton
+Leyh
+Leydecker
+Leyda
+Lexer
+Lewi
+Lewars
+Levreau
+Levra
+Levielle
+Levian
+Leveto
+Leversee
+Levers
+Leverone
+Leverance
+Levendoski
+Levee
+Levatino
+Levans
+Levandofsky
+Leuze
+Leutwiler
+Leuthe
+Leuhring
+Leuga
+Leuckel
+Leuasseur
+Lettsome
+Lettiere
+Letscher
+Letender
+Letchaw
+Leta
+Lestrange
+Lestourgeon
+Lestor
+Leston
+Lessner
+Lessmann
+Lessly
+Lespedes
+Leso
+Lesneski
+Leskovar
+Leskovac
+Lese
+Lesco
+Lesches
+Lesa
+Lerra
+Lerper
+Lerow
+Lero
+Lermon
+Lepretre
+Lepre
+Leppink
+Lepke
+Lepez
+Lepetich
+Leopardi
+Leonpacher
+Leonick
+Leonberger
+Leomiti
+Leny
+Lenski
+Lenorud
+Lenort
+Lennis
+Lennart
+Lennan
+Lenling
+Lenke
+Lenigan
+Lenhoff
+Lenharr
+Leners
+Lendt
+Lendor
+Lendo
+Lenczyk
+Lench
+Lenberg
+Lemoyne
+Lemmonds
+Lemmings
+Lemish
+Lemear
+Lembcke
+Lemansky
+Lemans
+Lellig
+Lekey
+Lekberg
+Lekan
+Lek
+Lejman
+Leitzinger
+Leithiser
+Leiper
+Leinwand
+Leimkuhler
+Leimberger
+Leilich
+Leigland
+Leichtenberge
+Leiberton
+Leho
+Lehning
+Lehneis
+Lehmer
+Lehenbauer
+Lehberger
+Legrotte
+Legro
+Legra
+Legat
+Legall
+Lefurgy
+Leflores
+Leffers
+Leffelman
+Lefeld
+Lefaver
+Leetham
+Leesman
+Leeker
+Leehan
+Leeber
+Ledsinger
+Ledermann
+Ledenbach
+Ledee
+Led
+Lecznar
+Leckband
+Lechleidner
+Lechelt
+Lecato
+Lecaros
+Lecain
+Lebroke
+Lebold
+Leblane
+Lebitski
+Lebish
+Leberte
+Lebedeff
+Lebby
+Lebaugh
+Lebarge
+Leavigne
+Leaven
+Leasor
+Leasher
+Leash
+Leanza
+Leanen
+Leaird
+Leahman
+Leadford
+Lazusky
+Lazurek
+Lazott
+Lazio
+Lazier
+Lazich
+Lazewski
+Lazares
+Layva
+Layell
+Laycox
+Lawsky
+Lawrentz
+Lawis
+Lawford
+Lawcewicz
+Lawbaugh
+Lawary
+Lawal
+Lavongsar
+Lavgle
+Lavezzo
+Lavelli
+Lave
+Lavani
+Lavander
+Lavagnino
+Lavadera
+Lautieri
+Lautaret
+Lausell
+Lauschus
+Laurole
+Lauretta
+Laureno
+Laureles
+Laurance
+Launiere
+Laundree
+Lauigne
+Laughon
+Laugen
+Laudeman
+Laudadio
+Lauckner
+Lauchaire
+Lauby
+Laubersheimer
+Latus
+Latourrette
+Latos
+Laton
+Lathrum
+Lather
+Lathe
+Latendresse
+Late
+Latassa
+Latam
+Lat
+Lastella
+Lassetter
+Laskosky
+Laskoskie
+Lasin
+Lasik
+Lashlee
+Lashier
+Laselle
+Laschinger
+Lascaro
+Lasane
+Lasagna
+Lasage
+Larusch
+Larrosa
+Larriviere
+Larralde
+Larr
+Larowe
+Larousse
+Larotta
+Laroia
+Laroe
+Larmett
+Larman
+Larkan
+Largena
+Laregina
+Lardone
+Larcom
+Larche
+Larbie
+Larbi
+Larason
+Laranjo
+Laragy
+Laraby
+Larabell
+Larabel
+Lapuerta
+Lappinga
+Lappi
+Laport
+Lapinta
+Lapila
+Laperuta
+Lapere
+Laper
+Lapek
+Lapari
+Lapalme
+Laorange
+Lanze
+Lanzarotta
+Lantry
+Lantgen
+Lantelme
+Lanteigne
+Lansey
+Lansberg
+Lannier
+Lannen
+Lanna
+Lankster
+Lanie
+Langrum
+Langness
+Langmo
+Langlitz
+Langi
+Langholdt
+Langhans
+Langgood
+Langanke
+Lanfor
+Lanen
+Laneaux
+Landu
+Landruth
+Landrie
+Landreville
+Landres
+Landquist
+Landolf
+Landmark
+Landini
+Landevos
+Landenberger
+Landan
+Lancz
+Lamudio
+Lampsas
+Lampl
+Lampinen
+Lamphiear
+Lampel
+Lamoree
+Lamoreau
+Lamoore
+Lamontagna
+Lammy
+Lammel
+Lamison
+Laming
+Lamie
+Lamia
+Lameda
+Lambuth
+Lambertus
+Lambermont
+Lamartina
+Lamango
+Lamaack
+Lalinde
+Lalich
+Lale
+Lakowski
+Lakhan
+Lajoye
+Lajoy
+Laios
+Lahne
+Laham
+Laguire
+Lagrenade
+Lagore
+Lagoo
+Lagonia
+Lagoni
+Laglie
+Laggan
+Lagesse
+Lagerstedt
+Lagergren
+Lagatta
+Lagard
+Lagant
+Lagamba
+Lagadinos
+Lafuze
+Lafrate
+Laforey
+Lafoon
+Lafontain
+Laflam
+Laffer
+Lafevre
+Lafemina
+Lafantano
+Laface
+Laessig
+Laehn
+Ladt
+Ladouce
+Ladonne
+Lado
+Ladika
+Ladick
+Ladebauche
+Lacz
+Lacusky
+Lacovara
+Lackett
+Lackage
+Lachino
+Lachiatto
+Lacharite
+Lacerenza
+Lacek
+Lacau
+Lacatena
+Lacaille
+Labovitch
+Labounta
+Labombar
+Laboissonnier
+Labo
+Labitan
+Labier
+Labeots
+Labarriere
+Labaro
+Labarbara
+Laatsch
+Laasaga
+Laake
+Kyseth
+Kypuros
+Kyper
+Kyner
+Kwilosz
+Kvzian
+Kvoeschen
+Kveton
+Kvek
+Kveen
+Kvaternik
+Kuziel
+Kuypers
+Kuykendoll
+Kuwana
+Kuwada
+Kutzer
+Kuty
+Kutlu
+Kuti
+Kutchie
+Kuszynski
+Kussmaul
+Kussel
+Kusnic
+Kusner
+Kusky
+Kushaney
+Kurzinski
+Kurtti
+Kurshuk
+Kurr
+Kurokawa
+Kurns
+Kuretich
+Kurasz
+Kurant
+Kura
+Kur
+Kupihea
+Kupferberg
+Kupersmith
+Kupchinsky
+Kunter
+Kunkleman
+Kuniyoshi
+Kunimitsu
+Kunich
+Kundanani
+Kunau
+Kummerow
+Kumlander
+Kumfer
+Kuman
+Kumalaa
+Kum
+Kulseth
+Kulbeth
+Kulbacki
+Kulback
+Kukura
+Kukler
+Kuklenski
+Kukauskas
+Kukahiko
+Kujat
+Kuiz
+Kuitu
+Kuick
+Kuhry
+Kuhlenschmidt
+Kuffa
+Kuepfer
+Kuehnhold
+Kuechler
+Kudro
+Kudrle
+Kuczma
+Kuckens
+Kuciemba
+Kuchinski
+Kuchem
+Kubley
+Kubler
+Kubesh
+Kubeck
+Kubasch
+Kub
+Kuanoni
+Krzewinski
+Krzesinski
+Krzan
+Kryston
+Krystek
+Krynicki
+Krylo
+Kruzel
+Kruyt
+Kruszewski
+Krusor
+Kruskie
+Krushansky
+Krush
+Kruppenbacher
+Krupinsky
+Krumroy
+Krumbein
+Krumbach
+Krukiel
+Kruizenga
+Kruis
+Kruiboesch
+Kruebbe
+Krucke
+Krotine
+Krostag
+Kropff
+Kropfelder
+Kroninger
+Kronau
+Krome
+Krolick
+Krokus
+Krog
+Krofta
+Krofft
+Kroesing
+Krochmal
+Krobath
+Krnach
+Krivanec
+Kristofferson
+Kristof
+Kristan
+Krissie
+Kriskovich
+Kriske
+Krishun
+Krishnamurthy
+Krishman
+Krinov
+Kriek
+Kriegshauser
+Krewer
+Kreutzbender
+Kreusch
+Kretzinger
+Kressler
+Kressin
+Kressierer
+Kresky
+Krepp
+Krenzke
+Krenning
+Krenik
+Kremple
+Kremmel
+Kremen
+Krejcik
+Kreissler
+Kreinhagen
+Krehel
+Kreese
+Krawitz
+Kravetsky
+Kravets
+Kravec
+Krausse
+Krausmann
+Krauel
+Kratowicz
+Kratchman
+Krasnici
+Krasnansky
+Kraskouskas
+Krasinski
+Kranwinkle
+Kranock
+Kramarczyk
+Krallman
+Krallis
+Krakowiak
+Krakauer
+Krainbucher
+Kraig
+Kraichely
+Krahulec
+Krahe
+Krah
+Kragt
+Kraetsch
+Krabel
+Krabbenhoft
+Kraasch
+Kraack
+Kozlovsky
+Kozlik
+Koziak
+Kozeyah
+Kozan
+Kowitz
+Kowalke
+Kowalec
+Koves
+Kovalaske
+Kovacik
+Koutras
+Koussa
+Kousonsavath
+Kounthong
+Kounthapanya
+Kounovsky
+Kounkel
+Kounick
+Koulavongsa
+Koulalis
+Kotyk
+Kotur
+Kottraba
+Kottlowski
+Kotterna
+Kotschevar
+Kotonski
+Kotlar
+Kotheimer
+Kotey
+Koterba
+Koteras
+Kotarski
+Kotaki
+Kosuta
+Kostrzewa
+Kostiv
+Kosters
+Kossey
+Kossen
+Kossak
+Kososky
+Kosorog
+Koso
+Koslan
+Kosiorek
+Koshi
+Koscielniak
+Kosareff
+Korzyniowski
+Korzybski
+Korynta
+Korwin
+Korwatch
+Kortemeier
+Korst
+Korsmeyer
+Korslund
+Koroch
+Kornn
+Kornfield
+Kornblatt
+Korkmas
+Koritko
+Korinta
+Koria
+Korewdit
+Kores
+Korenek
+Kordys
+Kordowski
+Kordiak
+Korbin
+Kopsho
+Koppy
+Kopke
+Kopin
+Kopicko
+Kopiasz
+Koperski
+Kopay
+Kopatz
+Kopan
+Koosman
+Koong
+Koolman
+Kool
+Konty
+Konow
+Konopski
+Konma
+Konishi
+Konger
+Konetchy
+Kone
+Konderla
+Konczewski
+Konarik
+Komula
+Kominski
+Komada
+Koma
+Kolwyck
+Kolupke
+Koltz
+Kolts
+Kolppa
+Koloc
+Kollross
+Kollos
+Kolkman
+Kolkhorst
+Kolikas
+Kolic
+Kolbusz
+Kolassa
+Kol
+Kokubun
+Kokoszka
+Kokko
+Kokenge
+Koitzsch
+Koiner
+Kohus
+Kohles
+Kohel
+Koguchi
+Kofoot
+Koers
+Koenitzer
+Koeninger
+Koenigsberg
+Koener
+Koenemund
+Koelbel
+Koehring
+Koeck
+Kody
+Kodera
+Koczwara
+Kocieda
+Kochkodin
+Kochen
+Kochanek
+Kobylski
+Kobylarz
+Kobylarczyk
+Kobold
+Knyzewski
+Knupke
+Knudsvig
+Knowiton
+Knowell
+Knous
+Knotowicz
+Knorp
+Knoflicek
+Knoeppel
+Knoepke
+Knoell
+Knoechel
+Knodel
+Knockaert
+Knobler
+Kniola
+Knill
+Knilands
+Kniesel
+Kniceley
+Kneuper
+Knetsch
+Kneser
+Knerien
+Knellinger
+Kneefe
+Knazs
+Knatt
+Knapko
+Knapick
+Knape
+Knap
+Knake
+Kmiotek
+Kment
+Kmatz
+Kman
+Klyn
+Klute
+Kluse
+Klumph
+Klukken
+Klukan
+Kluemper
+Kluber
+Klosky
+Kloppenburg
+Klonowski
+Klomp
+Klohs
+Klohe
+Kloeppel
+Kloeker
+Kloefkorn
+Kloeck
+Klobucar
+Kljucaric
+Klitzner
+Klitsch
+Kliskey
+Klinski
+Klinnert
+Klinich
+Klingner
+Klingenberger
+Klingberg
+Klingaman
+Klimo
+Klimavicius
+Klickman
+Klicka
+Klez
+Klevjer
+Klette
+Kletschka
+Kless
+Kleppen
+Klenovich
+Kleintop
+Kleinsasser
+Kleinfeld
+Kleifgen
+Kleid
+Kleftogiannis
+Kleefisch
+Kleck
+Klebes
+Klear
+Klawuhn
+Klawinski
+Klavon
+Klavetter
+Klarin
+Klappholz
+Klande
+Klancnik
+Klan
+Klamn
+Klamert
+Klaja
+Klaich
+Klafehn
+Klabunde
+Kjolseth
+Kjergaard
+Kjellsen
+Kjellman
+Kjeldgaard
+Kizzia
+Kizior
+Kivela
+Kitty
+Kitthikoune
+Kittelman
+Kitelinger
+Kitcher
+Kitchenman
+Kitanik
+Kisro
+Kisielewski
+Kiryakoza
+Kirsopp
+Kirshman
+Kirlin
+Kirkness
+Kirkling
+Kirkconnell
+Kirgan
+Kirchmann
+Kirchherr
+Kirchberg
+Kirchbaum
+Kirberger
+Kiracofe
+Kipple
+Kip
+Kious
+Kintopp
+Kintigh
+Kinsolving
+Kinsky
+Kinlin
+Kinlecheeny
+Kingwood
+Kingson
+Kinds
+Kindregan
+Kinderman
+Kinde
+Kimminau
+Kimbal
+Kilver
+Kiltie
+Kilstofte
+Kilogan
+Kilness
+Kilner
+Kilmister
+Killoren
+Killius
+Kilimnik
+Kilichowski
+Kildare
+Kiko
+Kijak
+Kiili
+Kihlstrom
+Kietzer
+Kiesser
+Kierzewski
+Kienbaum
+Kienast
+Kieke
+Kieck
+Kiebala
+Kiddle
+Kickel
+Kichline
+Kibbler
+Kiani
+Khubba
+Khora
+Khokher
+Khn
+Khlok
+Khilling
+Khensamphanh
+Khemmanivong
+Khazdozian
+Khazaleh
+Khauv
+Khairallah
+Kezele
+Keyon
+Keyl
+Kew
+Kevwitch
+Kevorkian
+Keveth
+Kevelin
+Kevan
+Keuper
+Ketzler
+Kettinger
+Ketterl
+Ketteringham
+Kettenring
+Ketchersid
+Kessans
+Kesey
+Kesek
+Kertzman
+Kertels
+Kerst
+Kerper
+Kernodle
+Kernighan
+Kernagis
+Kermes
+Kerens
+Kercheff
+Kerce
+Kerans
+Keppner
+Kepke
+Kepani
+Keovongxay
+Keoghan
+Keodalah
+Keobaunleuang
+Kenzie
+Kenson
+Kenoyer
+Kenouo
+Kennie
+Kenngott
+Kennaugh
+Kenik
+Keney
+Kenekham
+Kenealy
+Kendziora
+Kendal
+Kenaga
+Kempster
+Kemps
+Kempon
+Kempkens
+Kemmeries
+Kemerly
+Keltt
+Kellywood
+Kellish
+Kellem
+Keliipaakaua
+Kelau
+Keks
+Keisacker
+Keis
+Keinonen
+Keilholz
+Keilholtz
+Keihl
+Kehres
+Keetch
+Keetan
+Keet
+Keeser
+Keenom
+Keeman
+Keehner
+Keehan
+Kedra
+Kedia
+Kecskes
+Kecker
+Kebede
+Kebe
+Keba
+Keaty
+Keaten
+Keaser
+Kearsey
+Kearn
+Kazunas
+Kazimi
+Kazar
+Kazabi
+Kaza
+Kayat
+Kayastha
+Kawski
+Kawell
+Kawczynski
+Kawaiaea
+Kave
+Kavaney
+Kaut
+Kaushal
+Kausch
+Kauo
+Kaumans
+Kaui
+Kauder
+Kaucher
+Kaua
+Katzmann
+Katzaman
+Katterjohn
+Kattaura
+Katsaounis
+Katoh
+Katke
+Katis
+Katin
+Katie
+Kathleen
+Kathel
+Kataoka
+Kaszton
+Kaszinski
+Kasula
+Kasuba
+Kastens
+Kaspari
+Kasmarek
+Kasky
+Kashner
+Kasen
+Kasemeier
+Kasee
+Kasal
+Karz
+Karwowski
+Karstensen
+Karroach
+Karro
+Karrels
+Karpstein
+Karpe
+Karoly
+Karnath
+Karnas
+Karlinsky
+Karlgaard
+Kardux
+Karangelen
+Karamchandani
+Karagiannes
+Karageorge
+Karabin
+Kar
+Kapsner
+Kapperman
+Kappelmann
+Kapler
+Kapiloff
+Kapetanos
+Kanzenbach
+Kanwar
+Kantis
+Kantah
+Kanosh
+Kanoon
+Kanniard
+Kannan
+Kanjirathinga
+Kangleon
+Kaneta
+Kanekuni
+Kanealii
+Kand
+Kanakares
+Kamstra
+Kamradt
+Kampner
+Kamna
+Kammerzell
+Kamman
+Kamiya
+Kaminska
+Kamensky
+Kamber
+Kallhoff
+Kallfelz
+Kalley
+Kallestad
+Kallal
+Kalista
+Kalhorn
+Kalenak
+Kaldahl
+Kalberg
+Kalandek
+Kalan
+Kalamaras
+Kalafarski
+Kalaf
+Kakowski
+Kakeh
+Kakani
+Kajder
+Kaja
+Kaines
+Kaiktsian
+Kaid
+Kahookele
+Kahoohalphala
+Kahley
+Kahao
+Kahalehoe
+Kahal
+Kahae
+Kagimoto
+Kaewprasert
+Kaemingk
+Kadow
+Kadelak
+Kaczka
+Kacvinsky
+Kacprowski
+Kachmarsky
+Kabzinski
+Kabus
+Kabir
+Kabigting
+Kabala
+Kabacinski
+Kababik
+Kaarlela
+Kaanana
+Kaan
+Kaak
+Kaai
+Ka
+Juvenal
+Justian
+Juste
+Justak
+Jurries
+Jurney
+Jurkovich
+Jurist
+Jurin
+Jurgen
+Juray
+Junod
+Junkersfeld
+Junick
+Jumbo
+Julsrud
+Julitz
+Juliana
+Jukich
+Juengling
+Juen
+Juelich
+Judie
+Jubyna
+Jubran
+Jubeh
+Juback
+Juba
+Juanico
+Joynson
+Joyne
+Jover
+Journot
+Joto
+Jotblad
+Josic
+Jorrisch
+Jordt
+Jording
+Jondrow
+Jonah
+Jome
+Jollimore
+Joline
+Jolina
+Joler
+Joki
+Johnting
+Johnstonbaugh
+Johnikins
+Johniken
+Johe
+Johansing
+Johal
+Joganic
+Joerger
+Joelson
+Joehnck
+Jody
+Jodha
+Joanis
+Jirsa
+Jirak
+Jira
+Jingst
+Jhingree
+Jhanson
+Jews
+Jestis
+Jessica
+Jeskie
+Jesiolowski
+Jesenovec
+Jeschon
+Jermeland
+Jerkin
+Jericho
+Jerger
+Jergen
+Jerding
+Jepko
+Jens
+Jenovese
+Jennkie
+Jenderer
+Jenab
+Jempty
+Jemmings
+Jelome
+Jellings
+Jelden
+Jelarde
+Jeffryes
+Jeffirs
+Jedan
+Jecmenek
+Jecklin
+Jeck
+Jeanquart
+Jeanphilippe
+Jeannoel
+Jeanette
+Jeancy
+Jaysura
+Javis
+Javers
+Javed
+Jave
+Jaussen
+Jauhar
+Jastremski
+Jastrebski
+Jasmann
+Jaskolka
+Jasko
+Jaskiewicz
+Jasica
+Jasch
+Jarriett
+Jaroski
+Jarnutowski
+Jarmin
+Jaremka
+Jarema
+Jarels
+Jarecke
+Jarding
+Jardel
+Japak
+Janysek
+Janway
+Janowiec
+Janow
+Janofsky
+Janoff
+Jannise
+Jannett
+Jankoff
+Janeiro
+Jana
+Jaminet
+Jami
+Jamgochian
+Jamesson
+Jamer
+Jamel
+Jamason
+Jalovel
+Jalkut
+Jakubov
+Jaksic
+Jaksch
+Jakiela
+Jaji
+Jaiyesimi
+Jahosky
+Jahoda
+Jahaly
+Jagiello
+Jaggie
+Jafek
+Jafari
+Jae
+Jadoo
+Jaculina
+Jacquin
+Jacquelin
+Jacobsohn
+Jacobovits
+Jackso
+Jacksits
+Jackosn
+Jackett
+Jacinthe
+Jabbie
+Jabaut
+Jabali
+Jaarda
+Izak
+Izaguine
+Iwasko
+Iwashita
+Ivrin
+Ivener
+Iveans
+Ivancic
+Iuchs
+Itnyre
+Istorico
+Isiminger
+Isgur
+Isgro
+Isenbarger
+Iseman
+Isebrand
+Isaksen
+Isagba
+Isacson
+Isaack
+Irr
+Ironhorse
+Irigoyen
+Ireson
+Ipsen
+Iossa
+Inzano
+Introini
+Insognia
+Inserra
+Inostraza
+Innerst
+Innella
+Innarelli
+Innamorato
+Inkavesvanitc
+Ingvolostad
+Inguardsen
+Ingran
+Ingrahm
+Ingraffea
+Ingleton
+Inghem
+Ingersol
+Ingargiolo
+Inferrera
+Iner
+Induddi
+Indermuehle
+Indeck
+Indal
+Incomstanti
+Incera
+Incarnato
+Inbody
+Inabnit
+Imming
+Immerman
+Immediato
+Imholte
+Imeson
+Imbruglia
+Imbrock
+Imbriale
+Imbrenda
+Imam
+Imada
+Iltzsch
+Illovsky
+Illich
+Illas
+Illar
+Iliffe
+Ilg
+Ilarraza
+Ilaria
+Ilalio
+Ikzda
+Ikkela
+Ikenberry
+Ikemoto
+Ikemire
+Ikeard
+Ihnen
+Ihenyen
+Iheme
+Igus
+Iguina
+Ignoria
+Igles
+Igbinosun
+Ifie
+Ifft
+Ifeanyi
+Ifantides
+Iennaco
+Idrovo
+Idriss
+Idiart
+Ickert
+Icardo
+Ibric
+Ibdah
+Ibbotson
+Ibasitas
+Iarussi
+Iara
+Iannalo
+Iamiceli
+Iacuzio
+Iacobucci
+Iacobelli
+Hysquierdo
+Hyske
+Hydzik
+Hyberger
+Hyatte
+Huysman
+Huyna
+Hutyra
+Huttman
+Huttar
+Huter
+Husul
+Hustedt
+Hussy
+Hussong
+Hussian
+Huski
+Hushon
+Husein
+Husaini
+Hurtubise
+Hurta
+Hurni
+Hurme
+Hupy
+Huppenbauer
+Hunze
+Hunson
+Huner
+Hundertmark
+Hunderlach
+Humston
+Hummert
+Huminski
+Humerick
+Humbard
+Hulzing
+Hulshoff
+Hulmes
+Hukle
+Hujer
+Huitink
+Huirgs
+Hugus
+Huguet
+Hugghis
+Huffstutter
+Huerto
+Huertes
+Huenergardt
+Huemmer
+Huelle
+Huehn
+Huebsch
+Hudok
+Hudnut
+Hudlow
+Hudlin
+Hudes
+Huddy
+Huckabone
+Huckabaa
+Hubsch
+Hubl
+Hubertz
+Htwe
+Hsy
+Hrycko
+Hrna
+Hric
+Hribal
+Hrcka
+Hrbacek
+Hranchak
+Hradecky
+Hoysock
+Hoyne
+Hoylton
+Hoyal
+Hoxsie
+Howlingwolf
+Howett
+Howarter
+Hovnanian
+Hovard
+Hovantzi
+Hovanes
+Houzah
+Houtkooper
+Housner
+Housemate
+Hourihan
+Houltberg
+Houghtelling
+Houey
+Houchard
+Houben
+Hotter
+Hotten
+Hottell
+Hotek
+Hosoi
+Hosner
+Hosle
+Hoskyns
+Hoskey
+Hoshino
+Hosfield
+Hortein
+Horseford
+Horse
+Horridge
+Hornshaw
+Horns
+Hornlein
+Hornig
+Horneff
+Hormuth
+Horimoto
+Horesco
+Horenstein
+Horelick
+Hore
+Horbert
+Horabik
+Hoppenrath
+Hoppa
+Hopfauf
+Hoosock
+Hool
+Hoogheem
+Hoogendoorn
+Hoo
+Honus
+Honold
+Honokaupu
+Honigsberg
+Hongisto
+Hongeva
+Hones
+Honegger
+Hondros
+Hondel
+Honchul
+Honch
+Homza
+Homsey
+Homrighaus
+Hommer
+Homiak
+Homby
+Homans
+Holznecht
+Holzmiller
+Holzhueter
+Holzboog
+Holtmeier
+Holtmann
+Holthouse
+Holthoff
+Holtham
+Holtgrefe
+Holstad
+Holshovser
+Holquist
+Holmers
+Hollyday
+Hollo
+Hollner
+Hollinghurst
+Holleyman
+Hollett
+Hollerud
+Hollering
+Hollembaek
+Hollarn
+Hollamon
+Hollack
+Holihan
+Holibaugh
+Holgersen
+Holdy
+Holdgrafer
+Holdcraft
+Holdbrook
+Holcroft
+Holch
+Hokula
+Hokett
+Hojeij
+Hojczyk
+Hoivik
+Hoiseth
+Hoinacki
+Hohnson
+Hohney
+Hohmeier
+Hohm
+Hohlstein
+Hogstrum
+Hogon
+Hoglan
+Hogenmiller
+Hogains
+Hoga
+Hofstra
+Hofstadter
+Hofhine
+Hoffpavir
+Hoeser
+Hoerig
+Hoerger
+Hoelzel
+Hoelter
+Hoeller
+Hoek
+Hoehl
+Hoefflin
+Hoeffer
+Hodosy
+Hodnicki
+Hodermarsky
+Hodd
+Hockley
+Hochstine
+Hochfelder
+Hobstetter
+Hoblit
+Hobin
+Hoberek
+Hobb
+Hnot
+Hlywa
+Hlastala
+Hjermstad
+Hizkiya
+Hitzfelder
+Hiteman
+Hitchko
+Hitchingham
+Hissom
+Hismith
+Hiske
+Hirte
+Hirschmann
+Hirose
+Hirezi
+Hipsley
+Hippley
+Hipol
+Hintergardt
+Hinokawa
+Hinely
+Hindsman
+Hindmarsh
+Hinderaker
+Hindall
+Hinckson
+Hinajosa
+Himmelsbach
+Himmelright
+Hilyar
+Hilvers
+Hilu
+Hiltunen
+Hiltebeitel
+Hilsgen
+Hilovsky
+Hilo
+Hilmer
+Hillseth
+Hillered
+Hilleman
+Hillbrant
+Hillabush
+Hilla
+Hilkert
+Hilk
+Hildman
+Hilbner
+Hilbig
+Hilb
+Hila
+Hija
+Higy
+Hightshoe
+Higashida
+Hiens
+Hielscher
+Hidde
+Hidaka
+Hickley
+Hickingbotham
+Hickie
+Hiciano
+Hibble
+Hibbits
+Heziak
+Heynen
+Heykoop
+Heydenreich
+Heybrock
+Hevrin
+Hevessy
+Heugel
+Heuangvilay
+Hettes
+Hettenhausen
+Hetling
+Hetjonk
+Hethcox
+Hethcote
+Hetchman
+Hetcher
+Hesterly
+Hessman
+Hesselrode
+Hesselman
+Hesselbein
+Hesselbach
+Herzbrun
+Heryford
+Herwehe
+Hervol
+Hertle
+Herta
+Herskovic
+Hershnowitz
+Hershfield
+Herschaft
+Hersberger
+Herrud
+Herrnandez
+Herrlich
+Herritt
+Herrion
+Herrand
+Herran
+Herout
+Heroth
+Heronemus
+Hero
+Herny
+Hermus
+Herline
+Herley
+Hergenroeder
+Hergenreter
+Herena
+Herem
+Herek
+Hercman
+Heral
+Hequembourg
+Heppert
+Hepperly
+Heppel
+Heppding
+Henzler
+Hentrich
+Henter
+Hensle
+Hensdill
+Henschke
+Hennighausen
+Hennard
+Henkin
+Henges
+Henedia
+Hendson
+Hendsbee
+Hendrics
+Hendrickx
+Hencken
+Henchel
+Hencheck
+Hemsworth
+Hemry
+Hemperley
+Hemmig
+Hemmeter
+Hemmert
+Hemmelgarn
+Hemmeke
+Hemley
+Hemeyer
+Hemerly
+Hembre
+Hemans
+Hemanes
+Helwick
+Helvik
+Helphinstine
+Helphenstine
+Helowicz
+Helmert
+Helmen
+Helmbright
+Helliwell
+Helley
+Hellerman
+Hellenbrand
+Helferty
+Helfert
+Hekman
+Heitmuller
+Heitbrink
+Heisse
+Heisner
+Heir
+Heinzle
+Heinzerling
+Heino
+Heinig
+Heindl
+Heimerl
+Heimbuch
+Heilbrun
+Heilbron
+Heidtke
+Heidmann
+Heglund
+Heggins
+Heggestad
+Hegener
+Hegdahl
+Hefter
+Heffernen
+Heery
+Heebsh
+Hedrix
+Hedler
+Hedeiros
+Hedegaard
+Heddleson
+Heddins
+Hect
+Heckle
+Heckers
+Hebsch
+Hebrard
+Heberer
+Hebblethwaite
+Heaviland
+Heartley
+Hearston
+Heang
+Hean
+Heam
+Heagany
+Headlon
+Heading
+Hazouri
+Hazinski
+Hazekamp
+Hayword
+Haysbert
+Hayn
+Hayball
+Hawkings
+Havier
+Havermann
+Havekost
+Hauswald
+Haustein
+Hausteen
+Hauslein
+Hausher
+Haurin
+Hauptly
+Haulbrook
+Haukaas
+Haugaard
+Hauffe
+Hauben
+Hatzell
+Hatto
+Hattenbach
+Hatridge
+Hatlee
+Hathcox
+Hatchette
+Hatcherson
+Hatake
+Hassig
+Hasselvander
+Hasselkus
+Haslinger
+Haskamp
+Hashbarger
+Hasha
+Hasfjord
+Hasencamp
+Haseloff
+Haschke
+Hasbni
+Hasbell
+Hasak
+Harwin
+Harvley
+Harvilchuck
+Harvick
+Harutunian
+Hartzo
+Hartzheim
+Hartjen
+Hartgraves
+Hartgrave
+Hartgerink
+Hartenstein
+Harsy
+Harrisow
+Harrigton
+Harrellson
+Harralson
+Harrald
+Harradine
+Harraden
+Haroun
+Harnly
+Harnes
+Harnar
+Harnan
+Harnack
+Harlston
+Harlor
+Harleston
+Harkenreader
+Harkcom
+Harjochee
+Hargest
+Harges
+Harfert
+Harens
+Hardung
+Hardney
+Hardinson
+Hardigan
+Harby
+Harbus
+Harbough
+Harbottle
+Harbold
+Harary
+Haramoto
+Harader
+Harabedian
+Har
+Happney
+Happe
+Haper
+Hape
+Hanville
+Hanusey
+Hantzarides
+Hantula
+Hanstine
+Hansteen
+Hansson
+Hansrote
+Hansil
+Hanoharo
+Hanock
+Hannula
+Hanno
+Hannem
+Hanneken
+Hannegan
+Hanmore
+Hanisko
+Hanisco
+Hanify
+Hanhan
+Hanegan
+Handt
+Handshaw
+Handschumaker
+Handren
+Handlin
+Handing
+Handeland
+Hanagan
+Hanagami
+Hanafin
+Hanafan
+Hanacek
+Hamway
+Hampon
+Hamper
+Hamparian
+Hamor
+Hamontree
+Hamolik
+Hamnon
+Hamn
+Hammet
+Hammerstein
+Hammerstad
+Hammerlund
+Hammed
+Hammang
+Hameen
+Hamborsky
+Hamb
+Hamalak
+Hamai
+Halwood
+Halston
+Halpainy
+Halon
+Halmstead
+Halmick
+Hallstead
+Hallowich
+Hallio
+Hallie
+Hallerman
+Halleen
+Hallczuk
+Hallan
+Halgren
+Halechko
+Halcom
+Halbritter
+Halaliky
+Hal
+Hajdukiewicz
+Hait
+Haislett
+Hairster
+Hainsey
+Hainds
+Hailes
+Hagwell
+Hagon
+Haghighi
+Haggstrom
+Haggis
+Haggen
+Hageny
+Hagelgans
+Hagarty
+Hafenbrack
+Haessler
+Haessig
+Haerr
+Haener
+Haen
+Haeckel
+Hadson
+Hadland
+Hadian
+Haddaway
+Hackmeyer
+Hackethal
+Hackerd
+Hackenmiller
+Hackenbery
+Hacke
+Hackborn
+Hachette
+Habif
+Habermann
+Haberern
+Habbs
+Haakinson
+Haagensen
+Gzym
+Gyurko
+Gyllenband
+Gyaki
+Gwynes
+Gwenn
+Guzmdn
+Guziczek
+Guz
+Guyott
+Guyot
+Guyet
+Guttenberg
+Gutschow
+Gutreuter
+Gutrerrez
+Gutieres
+Gutiennez
+Guthorn
+Guthary
+Guterriez
+Gutenson
+Gussin
+Gushue
+Gusa
+Gurvine
+Gurtin
+Gurrad
+Gurne
+Guridi
+Gureczny
+Guralnick
+Gunzenhauser
+Gunthrop
+Gunkelman
+Gunagan
+Gun
+Gumphrey
+Gummersall
+Gumbert
+Gulnick
+Gullung
+Gullage
+Gulini
+Gulikers
+Guley
+Guldemond
+Gulde
+Gulbraa
+Gulati
+Guittennez
+Guitreau
+Guith
+Guitar
+Guirgis
+Guinle
+Guiltner
+Guilstorf
+Guillote
+Guillan
+Guilianelli
+Guilbe
+Guiffre
+Guiel
+Guidaboni
+Guiao
+Guialdo
+Guevana
+Guesman
+Guerrouxo
+Guerinot
+Gueretta
+Guenison
+Guenin
+Guempel
+Guemmer
+Guelpa
+Guelff
+Guelespe
+Guedesse
+Gudroe
+Gudat
+Guckes
+Gucciardi
+Gubser
+Gubitosi
+Gubernath
+Gubbins
+Guarracino
+Guarin
+Guariglio
+Guandique
+Guaman
+Gualdoni
+Guadalajara
+Grzywinski
+Grzywacz
+Grzyb
+Grzesiak
+Grygiel
+Gruzinsky
+Gruters
+Grusenmeyer
+Grupa
+Gruninger
+Grunin
+Grundon
+Gruhlke
+Gruett
+Gruesbeck
+Gruell
+Grueber
+Gruda
+Grubman
+Gruba
+Grovier
+Grothen
+Groszkiewicz
+Grossley
+Grossklaus
+Grosshans
+Grosky
+Groshek
+Grosenick
+Groscost
+Grosby
+Groombridge
+Gronvall
+Gromley
+Grollman
+Grohoske
+Groesser
+Groeber
+Grocott
+Grobstein
+Grix
+Grivna
+Gritsch
+Grit
+Gristede
+Grissam
+Grisostomo
+Grisom
+Grishan
+Grip
+Grinner
+Grinman
+Grines
+Grindel
+Grimlie
+Grimard
+Grillette
+Griggers
+Grigas
+Grigalonis
+Grigaliunas
+Grifin
+Griffins
+Griffes
+Griffel
+Grife
+Griesmeyer
+Griesi
+Griem
+Grham
+Grgurevic
+Greyovich
+Greydanus
+Greviston
+Gretzner
+Gretz
+Gretsch
+Greto
+Gresl
+Gresko
+Grengs
+Gremler
+Greist
+Greisser
+Greisiger
+Greiser
+Greiber
+Gregoroff
+Gregoreski
+Gregas
+Greenrose
+Greenlow
+Greenlees
+Greenfelder
+Greenen
+Greenbush
+Greeb
+Grebs
+Grebel
+Greaux
+Grdina
+Gravit
+Gravenstein
+Gravelin
+Grava
+Graul
+Graughard
+Graue
+Grat
+Grastorf
+Grassano
+Grasmuck
+Grashot
+Grasha
+Grappo
+Graper
+Granvil
+Granucci
+Grantier
+Granstaff
+Granroth
+Granizo
+Graniero
+Graniela
+Granelli
+Grandos
+Grandmont
+Gramza
+Graminski
+Gramberg
+Grahams
+Grago
+Graen
+Graefe
+Grae
+Gradle
+Graciani
+Graci
+Grabowiecki
+Grabauskas
+Gounder
+Gougeon
+Goudge
+Gouchie
+Gou
+Gottula
+Gottleber
+Gotthardt
+Gotowka
+Gotlib
+Gotimer
+Gothier
+Gothe
+Goswami
+Gostowski
+Gossin
+Gosserand
+Gossen
+Goshow
+Goshi
+Gosda
+Gosche
+Gorychka
+Gorri
+Gornikiewicz
+Gorlich
+Gorgo
+Gorglione
+Goretti
+Gorence
+Gorelik
+Goreczny
+Gordis
+Gorczynski
+Gorans
+Gootz
+Goosen
+Goonez
+Goolsbee
+Goolia
+Goodvin
+Goodpastor
+Goodgine
+Goodger
+Gooder
+Goodenberger
+Goodaker
+Goodacre
+Gonzolez
+Gonzaliz
+Gonsalues
+Gones
+Gone
+Gondran
+Gonda
+Gonazlez
+Gomzalez
+Gomey
+Gome
+Gomberg
+Golumski
+Goluba
+Goltry
+Goltra
+Golpe
+Golombecki
+Gollwitzer
+Gollogly
+Gollin
+Golkin
+Golk
+Goldware
+Goldrup
+Goldrich
+Goldhammer
+Goldhahn
+Goldfischer
+Goldfield
+Goldeman
+Goldak
+Golberg
+Golba
+Golanski
+Golabek
+Goick
+Gogocha
+Goglia
+Gogins
+Goetzke
+Goettman
+Goettig
+Goetjen
+Goeman
+Goeldner
+Goeken
+Goeden
+Godyn
+Godwyn
+Godown
+Godfray
+Goderich
+Gode
+Godde
+Goda
+Gockerell
+Gochnauer
+Gochie
+Gobrecht
+Gobeyn
+Gobern
+Gobea
+Gobbo
+Gobbi
+Gnagey
+Glugla
+Gluckman
+Gluc
+Glowski
+Glowka
+Glowinski
+Glow
+Glossner
+Gloff
+Gloe
+Glodich
+Gliwski
+Gliues
+Glise
+Glinkerman
+Glimp
+Glicher
+Glenny
+Glembocki
+Gleiss
+Gleichweit
+Gleghorn
+Glaviano
+Glauser
+Glaue
+Glaubke
+Glauberman
+Glathar
+Glasow
+Glashen
+Glasglow
+Glarson
+Glapion
+Glanden
+Glader
+Gladen
+Glacken
+Gjorven
+Gjokaj
+Gjesdal
+Gjelten
+Givliani
+Gitzlaff
+Gittere
+Gitlewski
+Gitchell
+Gissler
+Gisriel
+Gislason
+Girolami
+Girmazion
+Girellini
+Girauard
+Girardeau
+Girad
+Giove
+Gioriano
+Gionson
+Gioacchini
+Ginnetti
+Ginnery
+Ginanni
+Gillom
+Gillmer
+Gillerist
+Gillentine
+Gilhooley
+Gilfoy
+Gilespie
+Gildroy
+Gildore
+Gilcoine
+Gilarski
+Gihring
+Giggie
+Giessinger
+Gierling
+Gielstra
+Giehl
+Giegerich
+Giedlin
+Gieber
+Giebel
+Gidwani
+Gicker
+Gibes
+Gibbings
+Gibbard
+Gianopulos
+Gianola
+Giannell
+Giandelone
+Giancaspro
+Giancarlo
+Gian
+Giamichael
+Giagni
+Giacomazzi
+Giacoletti
+Giachino
+Ghramm
+Ghosten
+Ghiringhelli
+Ghiorso
+Ghil
+Ghia
+Gheza
+Ghekiere
+Gheewala
+Ghazvini
+Ghazi
+Ghazal
+Ghaor
+Ghane
+Ghanayem
+Ghamdi
+Gfroerer
+Geyette
+Gewinner
+Gewant
+Gevorkian
+Gevedon
+Geuder
+Getting
+Gettenberg
+Getschman
+Getachew
+Gestes
+Gesselli
+Geryol
+Gerych
+Gerty
+Gerton
+Gertken
+Gerster
+Gersch
+Gerpheide
+Geronime
+Gerondale
+Gerock
+Germinaro
+Germershausen
+Germer
+Gerlock
+Gerla
+Gerking
+Gerguson
+Geres
+Gerbs
+Gerbi
+Gerathy
+Gerardot
+Georgiana
+Georgales
+Geohagan
+Geoghan
+Geoffrey
+Genualdi
+Gentis
+Gennusa
+Gennaria
+Gennarelli
+Genin
+Genga
+Geng
+Geneseo
+Generous
+Generoso
+Genera
+Genberg
+Gemmel
+Gembe
+Gembarowski
+Gelzer
+Gelo
+Gellis
+Gellespie
+Gell
+Gelineau
+Gelger
+Geldrich
+Gelbach
+Geister
+Geissel
+Geisen
+Geiman
+Geils
+Gehrking
+Gehri
+Gehrett
+Gehred
+Gefroh
+Geerken
+Geelan
+Gedris
+Gedo
+Gechas
+Gecan
+Gebrayel
+Gebers
+Geasley
+Geanopulos
+Gdula
+Gbur
+Gazzillo
+Gazza
+Gazo
+Gaznes
+Gazdecki
+Gayoso
+Gayo
+Gaymes
+Gawlak
+Gavula
+Gavles
+Gaviria
+Gavinski
+Gavigan
+Gaves
+Gavell
+Gavalis
+Gautsch
+Gauron
+Gauntner
+Gaulzetti
+Gattie
+Gatski
+Gatch
+Gata
+Gastelun
+Gastellum
+Gastel
+Gasson
+Gassler
+Gasse
+Gasquet
+Gaspari
+Gasienica
+Gaseoma
+Gasch
+Garzone
+Garverick
+Garve
+Garthee
+Garrod
+Garriss
+Garrish
+Garraghty
+Garnet
+Garness
+Garnder
+Garlovsky
+Gariti
+Garich
+Garibaldo
+Garib
+Gargani
+Garfias
+Garff
+Garf
+Gares
+Garen
+Gardy
+Garder
+Garcelon
+Garced
+Garavelli
+Garala
+Garacci
+Ganze
+Gantewood
+Ganska
+Gannoe
+Ganji
+Ganja
+Ganibe
+Ganiban
+Ganguli
+Gangluff
+Gangadyal
+Gane
+Gandhy
+Gandarillia
+Gancio
+Gana
+Gamrath
+Gamewell
+Gamela
+Gamberini
+Gamberg
+Gambell
+Gambaiani
+Galvano
+Galva
+Galustian
+Galston
+Galstian
+Galson
+Gals
+Galon
+Galofaro
+Gallipo
+Gallery
+Galleno
+Gallegher
+Gallante
+Gallagos
+Gallaga
+Galjour
+Galinoo
+Galinol
+Galin
+Galietti
+Galhardo
+Galfayan
+Galetti
+Galetta
+Galecki
+Galauiz
+Galaska
+Galashaw
+Galarita
+Galanga
+Galacio
+Gailun
+Gailis
+Gaibler
+Gagon
+Gago
+Gagliardotto
+Gaetke
+Gaestel
+Gaekle
+Gadue
+Gades
+Gacusan
+Gacad
+Gabrel
+Gabouer
+Gabisi
+Gabino
+Gabbett
+Gabbay
+Gab
+Gaarsland
+Fyles
+Fventes
+Fusselman
+Fusik
+Fusi
+Fusha
+Fusca
+Furuyama
+Furubotten
+Furton
+Furrh
+Furne
+Furna
+Furlotte
+Furler
+Furkin
+Furfey
+Fure
+Furch
+Furay
+Fupocyupanqui
+Funderbunk
+Fundenberger
+Fulwiler
+Fulsom
+Fullwiler
+Fulliton
+Fulling
+Fuleki
+Fulda
+Fukuroku
+Fukada
+Fuhri
+Fuglsang
+Fugle
+Fugah
+Fuesting
+Fuents
+Fudacz
+Fucile
+Fuchser
+Frydman
+Fryday
+Fruusto
+Frutoz
+Frullate
+Fruchey
+Frossard
+Fross
+Froschheiser
+Froozy
+Fronduti
+Frondorf
+Fron
+Fromong
+Frometa
+Froiland
+Frohwein
+Frohock
+Froeliger
+Frodsham
+Fritzpatrick
+Frist
+Frisino
+Frisella
+Frischkorn
+Fringuello
+Frings
+Friling
+Frikken
+Frietsch
+Friest
+Friedstrom
+Friedhaber
+Friedenberg
+Friedeck
+Fridal
+Freytas
+Freydel
+Freudiger
+Freshley
+Frere
+Frenner
+Freniere
+Fremon
+Fremming
+Freme
+Freligh
+Freistuhler
+Freiser
+Freil
+Freifeld
+Freidkin
+Freidet
+Frehse
+Freguson
+Freerksen
+Freelon
+Freeley
+Freehoffer
+Freedland
+Fredrikson
+Fredric
+Fredline
+Fredicks
+Freddrick
+Frawkin
+Frauenkron
+Frati
+Franzeo
+Frantzich
+Frankina
+Frankford
+Frankenreiter
+Frankenfeld
+Franeo
+Frandeen
+Franculli
+Francolino
+Francoise
+Francisque
+Franciosa
+Francios
+Francione
+Franceski
+Franceschina
+Fram
+Fraine
+Fragassi
+Fracier
+Fraccola
+Frabotta
+Frabizio
+Fouyer
+Foux
+Foutain
+Fourre
+Fouracre
+Found
+Foules
+Foucha
+Fosso
+Fosser
+Fossa
+Fosburgh
+Forwood
+Fortado
+Forston
+Forsthoffer
+Forschner
+Forsch
+Fornkohl
+Fornerod
+Formhals
+Formey
+Formento
+Formato
+Forlani
+Forgy
+Forgach
+Fordon
+Forcino
+Forcell
+Forcade
+Forbish
+Forber
+Fontneau
+Fontelroy
+Fonteboa
+Fontanini
+Fonsecn
+Fondell
+Fon
+Follie
+Foller
+Folkins
+Folkens
+Folgar
+Foks
+Fogus
+Fogo
+Foerschler
+Foell
+Foecke
+Foderaro
+Foddrill
+Focks
+Flum
+Flugence
+Fluette
+Fluetsch
+Flueck
+Flournay
+Flotow
+Flota
+Florkowski
+Florestal
+Florance
+Floore
+Floerchinger
+Flodman
+Floch
+Flitton
+Flitt
+Flister
+Flinton
+Flinspach
+Flierl
+Flever
+Fleurissaint
+Fleurantin
+Flether
+Flennoy
+Fleitman
+Flegler
+Fleak
+Flautt
+Flaum
+Flasher
+Flaminio
+Fixari
+Fiumefreddo
+Fitzmier
+Fitzgerlad
+Fitzen
+Fittje
+Fitser
+Fitchette
+Fisichella
+Fisger
+Fischbein
+Fischang
+Fiscal
+Fisanick
+Firoozbakht
+Firlik
+Firkey
+Fiorenzi
+Fiora
+Finucan
+Finto
+Finona
+Finocan
+Finnley
+Finnin
+Finnila
+Finni
+Finnel
+Finne
+Finland
+Finkenbiner
+Finey
+Finders
+Filzen
+Filyan
+Filteau
+Filonuk
+Fillo
+Fillerup
+Filkey
+Filippides
+Filippello
+Filburn
+Filbrardt
+Filbey
+Filary
+Filarecki
+Filak
+Fijalkowski
+Figurelli
+Figone
+Figlioli
+Figlar
+Figary
+Figarsky
+Fiermonte
+Fierge
+Fiely
+Fieldstadt
+Fiedtkou
+Fiedorowicz
+Fiebich
+Fie
+Fidsky
+Fido
+Ficenec
+Feyler
+Fewless
+Feulner
+Feuerberg
+Fetui
+Fetrow
+Fesus
+Fesenbek
+Ferugson
+Ferster
+Ferrise
+Ferratt
+Ferratella
+Ferrarotti
+Ferrarini
+Ferrao
+Ferrandino
+Ferrall
+Ferracioli
+Feron
+Ferndez
+Fernandz
+Fermo
+Ferm
+Ferlic
+Ferjerang
+Feris
+Ferentz
+Fereday
+Ferdin
+Ferdico
+Ferderer
+Ferard
+Feramisco
+Fenti
+Fensel
+Fenoglio
+Fenoff
+Feno
+Fenniwald
+Fenger
+Fenceroy
+Felzien
+Felson
+Felsher
+Fellon
+Felli
+Fellhauer
+Fellenbaum
+Felleman
+Fellars
+Felks
+Felipa
+Felila
+Felico
+Felicione
+Felger
+Feldtman
+Feldner
+Feldker
+Feldhake
+Felciano
+Felcher
+Fekety
+Feindt
+Feinblatt
+Feilbach
+Feikles
+Feigh
+Feichtner
+Fehribach
+Fehnel
+Fehn
+Fegurgur
+Fego
+Fefer
+Feezor
+Feery
+Feerst
+Feeling
+Feekes
+Feduniewicz
+Feduccia
+Fedorka
+Fedoriw
+Fedorczyk
+Fedel
+Feddes
+Fedderly
+Fechtel
+Fecat
+Feazelle
+Feast
+Fearheller
+Fearen
+Feamster
+Fealy
+Fazzinga
+Fawell
+Favilla
+Favieri
+Favaron
+Favaro
+Faustman
+Faurot
+Faur
+Faulstick
+Faulstich
+Faulkes
+Faulkenbury
+Faulisi
+Faubus
+Fat
+Faster
+Fash
+Fasenmyer
+Fasci
+Fasbender
+Faruolo
+Farrin
+Farria
+Farrauto
+Farmsworth
+Farmar
+Farm
+Farlee
+Fariello
+Farid
+Farha
+Fardo
+Faraco
+Fantz
+Fanner
+Famy
+Famiano
+Fam
+Falu
+Faltz
+Falto
+Falson
+Fallie
+Fallick
+Falla
+Falknor
+Falkenthal
+Falis
+Falha
+Falge
+Falconeri
+Falcione
+Falchi
+Falb
+Falasco
+Falah
+Falack
+Falacco
+Faix
+Faisca
+Fairy
+Fairly
+Faigle
+Faichtinger
+Fahrenwald
+Fahrenbruck
+Fahner
+Fahlstedt
+Fagnoni
+Faglie
+Fagala
+Faehnle
+Fadri
+Fadei
+Facenda
+Fabus
+Fabroquez
+Fabello
+Fabeck
+Fabbozzi
+Ezernack
+Ezer
+Ezechu
+Ezdebski
+Eyubeh
+Eyermann
+Extine
+Expose
+Ewelike
+Evora
+Eviston
+Evertz
+Eversmann
+Everleth
+Evering
+Eveline
+Eveler
+Evanski
+Evanosky
+Evanoski
+Evanchyk
+Evanchalk
+Euton
+Euser
+Eurton
+Europe
+Ettl
+Ettison
+Etters
+Etoll
+Ethel
+Etchinson
+Esty
+Esteybar
+Estevane
+Esterson
+Esterling
+Estergard
+Estela
+Estaban
+Esshaki
+Essepian
+Esselman
+Essaid
+Essaff
+Esquiuel
+Esquerre
+Esquea
+Esposita
+Espenscheid
+Esparaza
+Esoimeme
+Esnard
+Eskuchen
+Eskelsen
+Eskeets
+Eskaran
+Eskaf
+Eshlerman
+Esenwein
+Escorza
+Escoe
+Escobeo
+Eschenbacher
+Eschenbach
+Eschborn
+Escarrega
+Escalet
+Esbensen
+Esannason
+Ervine
+Ervay
+Ertelt
+Erpenbach
+Ero
+Ernstrom
+Ernspiker
+Ernandez
+Ermogemous
+Ermita
+Erm
+Erlwein
+Erlanson
+Erixon
+Erice
+Erfert
+Ereth
+Erdmun
+Erdelt
+Erchul
+Ercek
+Erbentraut
+Erard
+Eracleo
+Equiluz
+Eppert
+Epperheimer
+Eppenger
+Epifano
+Eperson
+Enzenauer
+Entzi
+Entrup
+Entel
+Enote
+Enocencio
+Enny
+Ennist
+Ennels
+Ennaco
+Enkerud
+Enick
+Engwer
+Engleby
+Enget
+Engessor
+Engerman
+Engbretson
+Enfort
+Ends
+Endresen
+Endecott
+Encalade
+Emuka
+Emslander
+Emshoff
+Empleo
+Empfield
+Emperor
+Emo
+Emmrich
+Emlin
+Emigholz
+Emfield
+Emeru
+Emeche
+Emdee
+Emberlin
+Emberley
+Emberger
+Emayo
+Emanus
+Emami
+Elvert
+Elshair
+Elsensohn
+Elsbury
+Elsa
+Elroy
+Elquist
+Elofson
+Elmaghrabi
+Ellworths
+Ellifritt
+Ellies
+Elliem
+Ellerkamp
+Ellerbeck
+Ellenbee
+Ellena
+Ellebrecht
+Elldrege
+Ellanson
+Elko
+Elkayam
+Eliszewski
+Eliseo
+Elis
+Elion
+Elhosni
+Elhassan
+Elhaj
+Elhaddad
+Elgen
+Elgas
+Elgar
+Elg
+Elftman
+Elfering
+Elewa
+Eleveld
+Elefritz
+Elbogen
+Elbertson
+Elberson
+Elbahtity
+Elahi
+Ekstrum
+Eklov
+Ekis
+Ejide
+Eissinger
+Eirls
+Einfeldt
+Eilts
+Eilders
+Eilbert
+Eilbeck
+Eikmeier
+Eifler
+Eiesland
+Eichstadt
+Eichenmiller
+Eichenauer
+Eichelmann
+Ehr
+Ehorn
+Ehnis
+Ehmen
+Ehleiter
+Ehinger
+Ehiginator
+Ehigiator
+Egvirre
+Egure
+Eguizabal
+Ego
+Egidio
+Eggenberg
+Eggart
+Eget
+Egertson
+Egbe
+Efrati
+Eflin
+Eerkes
+Ee
+Edwads
+Edster
+Edralin
+Edmerson
+Edmeier
+Edleston
+Edlao
+Edith
+Edis
+Edeline
+Edeker
+Economus
+Economides
+Ecoffey
+Eckrote
+Eckmeyer
+Eckle
+Ecklar
+Eckis
+Echemendia
+Echavez
+Echaure
+Ebrani
+Ebo
+Ebilane
+Ebesugawa
+Eberting
+Ebersol
+Eberline
+Eberl
+Ebenstein
+Eben
+Ebbesen
+Ebach
+Easom
+Easlick
+Easker
+Easey
+Easdon
+Earman
+Earll
+Earlgy
+Earenfight
+Earehart
+Ealley
+Ealick
+Eagy
+Eafford
+Dziurawiec
+Dzierzanowski
+Dziegielewski
+Dziduch
+Dziadek
+Dzama
+Dyser
+Dys
+Dyreson
+Dymke
+Dyen
+Dwyar
+Dwornik
+Dwellingham
+Duxbury
+Duwhite
+Duverney
+Duvel
+Dutschmann
+Dutel
+Dute
+Dusak
+Durun
+Dursch
+Durrwachter
+Durousseau
+Durol
+Durig
+Durett
+Duresky
+Durelli
+Duree
+Dural
+Duraku
+Dupouy
+Duplin
+Duplesis
+Duplaga
+Dupaty
+Duonola
+Dunzelman
+Dunten
+Dunt
+Dunster
+Dunnahoo
+Dunmead
+Dunks
+Dunkentell
+Dunemn
+Duncker
+Dunckel
+Dunahoo
+Dummitt
+Dumez
+Dumag
+Dulberg
+Dulatre
+Dukhovny
+Dukeshire
+Dukeshier
+Duitscher
+Duitch
+Duh
+Dugmore
+Dughi
+Duffus
+Duffany
+Dufer
+Duesenberg
+Duerkson
+Duerkop
+Duenke
+Duel
+Dudleson
+Dudik
+Duderstadt
+Dudack
+Duchow
+Duchesney
+Duchatellier
+Ducceschi
+Ducayne
+Ducay
+Ducatelli
+Dubonnet
+Duberstein
+Dubej
+Dubeck
+Dubeau
+Dubbin
+Duban
+Duball
+Duartes
+Dsaachs
+Dryman
+Drybread
+Drumwright
+Drumheiser
+Drumgole
+Drullard
+Drue
+Drude
+Druckhammer
+Dru
+Drought
+Drossos
+Drossman
+Droski
+Drong
+Drones
+Dronen
+Droegmiller
+Drock
+Drisdelle
+Drinkall
+Drimmer
+Driggins
+Driesel
+Driere
+Drewski
+Dreps
+Dreka
+Dreith
+Dregrich
+Dreggs
+Drawy
+Drawec
+Dravland
+Drape
+Dramis
+Drainer
+Dragun
+Dragt
+Dragotta
+Dragaj
+Drafton
+Drafall
+Drader
+Draa
+Dozois
+Dozar
+Doyan
+Doxon
+Dowsett
+Dovenmuehler
+Douyon
+Douvier
+Douvia
+Douthart
+Doussan
+Dourado
+Doulani
+Douillet
+Dougharity
+Dougall
+Douet
+Dou
+Dotto
+Dottery
+Dotstry
+Doto
+Dotie
+Doswell
+Doskocil
+Doseck
+Dorweiler
+Dorvillier
+Dorvee
+Dortilla
+Dorsainvil
+Dorrian
+Dorpinghaus
+Dorph
+Dorosan
+Dornseif
+Dornhelm
+Dornellas
+Dorne
+Dornbos
+Dormanen
+Dormane
+Doriean
+Dorer
+Dorcent
+Dorat
+Dopf
+Dootson
+Doornbos
+Dooney
+Donten
+Dontas
+Donota
+Donohve
+Donning
+Donnellon
+Donne
+Donmore
+Donkor
+Donkervoet
+Donhoe
+Dongo
+Donelon
+Donchatz
+Donawa
+Donar
+Domnick
+Domkowski
+Domio
+Dominis
+Dominiquez
+Dominicus
+Dominico
+Domingus
+Domianus
+Domas
+Dolven
+Dolliver
+Doljac
+Doliveira
+Dolhon
+Dolgas
+Dolfay
+Dolcetto
+Dokuchitz
+Doino
+Doiel
+Doffing
+Doerflinger
+Doepner
+Doelling
+Dodich
+Doderer
+Dockray
+Dockett
+Docker
+Docimo
+Dobre
+Dobrasz
+Dobmeier
+Dobesh
+Dobberfuhl
+Dobb
+Dmitriev
+Dlobik
+Dlabaj
+Djuric
+Dizadare
+Divento
+Divan
+Diulio
+Ditti
+Dittbrenner
+Ditta
+Ditolla
+Ditchfield
+Distilo
+Distance
+Disponette
+Dispirito
+Dishinger
+Discon
+Disarufino
+Disabato
+Diruzzo
+Dirose
+Dirollo
+Dirado
+Dippery
+Dionisopoulos
+Diones
+Dinunzio
+Dinucci
+Dinovo
+Dinovi
+Dinola
+Dinho
+Dings
+Dinglasan
+Dingel
+Dinco
+Dimperio
+Dimoulakis
+Dimopoulos
+Dimmack
+Dimling
+Dimitriou
+Dimes
+Dilthey
+Dilox
+Dillworth
+Dillmore
+Dilligard
+Dilleshaw
+Dilgard
+Dilda
+Dilcher
+Dilchand
+Dikkers
+Diket
+Dikens
+Digrazia
+Digness
+Digiorgi
+Digiambattist
+Digesare
+Difiora
+Diffendal
+Diewold
+Dietsche
+Diestel
+Diesen
+Dien
+Diemoz
+Dielman
+Diegidio
+Diedricks
+Diebol
+Didlake
+Didamo
+Dickun
+Dickstein
+Dickirson
+Dickins
+Dicioccio
+Diciano
+Dichristopher
+Dicaro
+Dicara
+Dibrino
+Dibenedict
+Diamico
+Diak
+Diachenko
+Dhosane
+Dezell
+Dezayas
+Deyette
+Deyarmond
+Deyarmin
+Dewyer
+Dewulf
+Dewit
+Dewinne
+Dewaratanawan
+Devreese
+Devitto
+Devincenzi
+Devick
+Devey
+Devenecia
+Devel
+Deuschle
+Deuschel
+Deuman
+Deuermeyer
+Detz
+Deturenne
+Dettra
+Dettore
+Dettmering
+Dettmann
+Detterich
+Detorres
+Detlefs
+Detjen
+Detillier
+Dethomasis
+Detering
+Detar
+Desutter
+Destime
+Destephano
+Desrocher
+Desquare
+Desporte
+Desparrois
+Desort
+Desormo
+Desorbo
+Desolier
+Desmarias
+Desloge
+Deslaurier
+Desjardiws
+Desiyatnikov
+Desisles
+Desilvo
+Desiato
+Deshazior
+Desforges
+Deserres
+Deschomp
+Deschino
+Deschambeault
+Desautelle
+Desantigo
+Desan
+Deruso
+Derubeis
+Derriso
+Derricott
+Derrer
+Deroos
+Deroko
+Deroin
+Deroest
+Derobles
+Dernier
+Dermo
+Derkach
+Derizzio
+Deritis
+Derion
+Deriggi
+Dergurahian
+Dereu
+Derer
+Derenzis
+Derenthal
+Derensis
+Derendal
+Derenberger
+Deremiah
+Deraveniere
+Deramo
+Deralph
+Depsky
+Deprizio
+Deprince
+Deprez
+Depratt
+Depottey
+Depippo
+Depinho
+Depietro
+Depetris
+Deperte
+Depena
+Depaulis
+Depasse
+Depace
+Deonarian
+Deodato
+Denski
+Densieski
+Denoyelles
+Denofrio
+Denni
+Dennert
+Denna
+Deniken
+Denier
+Denice
+Denhartog
+Dench
+Dence
+Denburger
+Denafo
+Demyers
+Demulling
+Demuizon
+Demosthenes
+Demoney
+Demonett
+Demmon
+Demich
+Demian
+Demetris
+Demetree
+Demeris
+Demchok
+Dembosky
+Dembinski
+Dember
+Demauri
+Dematos
+Demasters
+Demarrais
+Demarini
+Demarc
+Demara
+Delvin
+Delveechio
+Delusia
+Deluney
+Deluccia
+Delre
+Delpiano
+Delosanglel
+Delosangeles
+Delon
+Delnegro
+Dellos
+Dellon
+Delling
+Dellibovi
+Dellasciucca
+Dellasanta
+Dellapina
+Dellajacono
+Dellagatta
+Dellaca
+Deliso
+Delinois
+Delilli
+Delilla
+Deliberato
+Delhomme
+Delguercio
+Delger
+Delgadilo
+Delfi
+Delfelder
+Deley
+Delevik
+Delettre
+Delessio
+Deleonardo
+Delellis
+Delehoy
+Delegeane
+Deldeo
+Delcine
+Delbusto
+Delbrune
+Delbrocco
+Delbo
+Delasko
+Delashaw
+Delasancha
+Delaremore
+Delaplane
+Delapenha
+Delanoche
+Delalla
+Delaguila
+Delaglio
+Dekuyper
+Dekort
+Dekorne
+Deklerk
+Dekine
+Dejoode
+Dejes
+Dejarme
+Dejager
+Deja
+Deischer
+Deir
+Deighton
+Deidrick
+Deida
+Deible
+Dehrer
+Dehombre
+Dehler
+Dehghani
+Dehan
+Dehaemers
+Degunya
+Deguise
+Degrella
+Degrazio
+Degrandpre
+Degori
+Degolyer
+Deglopper
+Deglanville
+Degado
+Defrates
+Defrancis
+Defranceschi
+Defouw
+Defiguero
+Defiglio
+Defide
+Defaria
+Deeters
+Dedominicis
+Dedo
+Dedier
+Dedek
+Deculus
+Decroo
+Decree
+Decourley
+Decomo
+Declouette
+Declet
+Declark
+Deckelman
+Dechart
+Dechamplain
+Decasanova
+Decardo
+Decardenas
+Decann
+Decaneo
+Debrita
+Debrie
+Debraga
+Debnar
+Debiew
+Debes
+Debenham
+Debello
+Debarba
+Deback
+Dearstyne
+Dearco
+Deanne
+Deanhardt
+Deamer
+Deaguero
+Daylong
+Daya
+Dawber
+Dawahoya
+Davydov
+Davtyan
+Davos
+Davirro
+Davidek
+Davide
+Davers
+Davensizer
+Davel
+Davda
+Dauzart
+Daurizio
+Dauila
+Daughetee
+Dauge
+Daufeldt
+Daudier
+Daubenmire
+Daty
+Datu
+Datte
+Dastoli
+Daste
+Dasso
+Daskam
+Dasinger
+Dasalia
+Daryanl
+Darvile
+Darsi
+Darsch
+Darrup
+Darnel
+Darm
+Darjean
+Dargenio
+Darey
+Dardashti
+Dardagnac
+Darbro
+Darbeau
+Daramola
+Daquip
+Dapvaala
+Danza
+Dantoni
+Dantes
+Danoski
+Danns
+Dannecker
+Danfield
+Danella
+Danczak
+Dancoes
+Damphousse
+Damoth
+Damoro
+Dammrich
+Dammad
+Damis
+Damerell
+Dambrozio
+Dama
+Daltorio
+Dalponte
+Dalomba
+Dalmida
+Dalmau
+Dallen
+Dalla
+Dalitz
+Dalio
+Dalhart
+Daleus
+Dalene
+Dalee
+Dalbeck
+Dalaq
+Dair
+Daimaru
+Daill
+Daichendt
+Dahood
+Dahlstedt
+Dahley
+Dahler
+Dagnone
+Dagnon
+Dagner
+Daggy
+Daer
+Dae
+Dadds
+Daddea
+Daddabbo
+Dad
+Dacres
+Dachs
+Dachelet
+Daber
+Czyrnik
+Czwakiel
+Czupryna
+Czubia
+Czosek
+Czernovski
+Czerno
+Czernik
+Czerniak
+Czekaj
+Czarniecki
+Cyler
+Cychosz
+Cuzzo
+Cuva
+Cutri
+Cutone
+Cutia
+Cutburth
+Cusworth
+Custa
+Cusmano
+Cushway
+Cushinberry
+Cusher
+Cushen
+Cushard
+Cusatis
+Curzi
+Curylo
+Curriere
+Currans
+Curra
+Curpupoz
+Curls
+Curleyhair
+Curella
+Cureau
+Curameng
+Cupe
+Cunningan
+Cunnane
+Cummisky
+Cummer
+Cumley
+Cumblidge
+Culotti
+Cullin
+Culajay
+Cujas
+Cuez
+Cuddihee
+Cudan
+Cuchiara
+Cuccinello
+Cucchiaro
+Cuartas
+Cuaresma
+Cuadro
+Csensich
+Cruthirds
+Cruthers
+Crutchev
+Crutch
+Crummedyo
+Crumlish
+Cruiz
+Cruey
+Cruel
+Croxford
+Croxen
+Crowin
+Croutch
+Croushorn
+Crotwell
+Crother
+Croslen
+Crookston
+Cronholm
+Cronauer
+Cromeens
+Crogier
+Croffie
+Crocitto
+Critzman
+Criton
+Critchelow
+Cristofaro
+Cristello
+Cristelli
+Crissinger
+Crispo
+Criqui
+Crickenberger
+Cressell
+Cresencio
+Creglow
+Creggett
+Creenan
+Creeley
+Credo
+Credille
+Crease
+Crawn
+Cravenho
+Cravatta
+Cration
+Crantz
+Cragar
+Cragan
+Cracolici
+Cracknell
+Craawford
+Craan
+Cozadd
+Coyier
+Cowser
+Cowns
+Cowder
+Covotta
+Covitt
+Covil
+Covarruvia
+Covarrubio
+Covarrubia
+Covar
+Cova
+Coutino
+Cousey
+Courtoy
+Courtad
+Couron
+Courneya
+Courie
+Couret
+Courchine
+Countis
+Counceller
+Cottillion
+Cottengim
+Cotroneo
+Cotreau
+Cotheran
+Cotey
+Coteat
+Cotant
+Coswell
+Costenive
+Costellowo
+Costeira
+Costanzi
+Cossaboon
+Cossaboom
+Cosimini
+Cosier
+Cosca
+Cosano
+Corvelli
+Corti
+Cortesi
+Corsilles
+Corsey
+Corseri
+Corron
+Corridoni
+Corrett
+Correo
+Corren
+Correau
+Corraro
+Corporon
+Corporal
+Corpeno
+Corolla
+Corolis
+Cornes
+Cornelson
+Cornea
+Cornacchio
+Cormican
+Cormia
+Coriz
+Coric
+Coriaty
+Coriano
+Corderman
+Cordel
+Corde
+Cordasco
+Corburn
+Corallo
+Coradi
+Coponen
+Coples
+Copier
+Copa
+Coopey
+Coonley
+Coomey
+Coolbrith
+Coolbeth
+Coolahan
+Cookey
+Coogen
+Cooey
+Cooch
+Conze
+Conzalez
+Contreros
+Contreres
+Contras
+Contraras
+Contopoulos
+Contofalsky
+Contino
+Consoli
+Consigli
+Conoly
+Connyer
+Conninghan
+Connette
+Connerty
+Connarton
+Conlans
+Conkrite
+Confrey
+Confair
+Coneys
+Conelly
+Conejo
+Condreay
+Condino
+Condell
+Condelario
+Concini
+Concilio
+Concho
+Conces
+Concepion
+Conceicao
+Conable
+Compres
+Compiseno
+Compeau
+Compean
+Comparoni
+Companie
+Compagna
+Comoletti
+Commes
+Comment
+Comeauy
+Colyott
+Columbres
+Colsch
+Colpaert
+Colpack
+Colorina
+Colopy
+Colonnese
+Colona
+Colomy
+Colombe
+Colomba
+Colmer
+Colly
+Collozo
+Collova
+Collora
+Collmeyer
+Collaco
+Colian
+Colglazier
+Colehour
+Colebrook
+Coldsmith
+Colden
+Colato
+Colasanti
+Colasamte
+Colarossi
+Colander
+Colaizzo
+Colaiacovo
+Coladonato
+Colacone
+Colabrese
+Cokins
+Cohoe
+Coho
+Cohlmia
+Cohagan
+Cogen
+Cofrancesco
+Cofran
+Codey
+Codeluppi
+Cocran
+Cocozza
+Cocoran
+Cocomazzi
+Cockrin
+Cockreham
+Cocking
+Cochis
+Cocherell
+Coccoli
+Cobio
+Cobane
+Coatley
+Coatie
+Coant
+Coaker
+Coachys
+Cmiel
+Clozza
+Cloughly
+Clothey
+Closovschi
+Closey
+Cloman
+Cloffi
+Cloepfil
+Clites
+Clinker
+Cleverly
+Cleve
+Clesen
+Clery
+Clerf
+Clemson
+Clemo
+Clemmon
+Clemmo
+Clemmey
+Cleark
+Clayter
+Clavey
+Clavelle
+Clausel
+Claud
+Claucherty
+Claton
+Clarson
+Clarendon
+Clarbour
+Clar
+Clap
+Clanin
+Clan
+Claman
+Clam
+Claes
+Civitello
+Civcci
+Civatte
+Civale
+Ciucci
+Cito
+Cisneroz
+Cislo
+Cisewski
+Cirioni
+Cirilli
+Cipullo
+Cippina
+Cipolone
+Cipolloni
+Cioni
+Cintra
+Cinkosky
+Cinalli
+Cimmiyotti
+Cimeno
+Cilva
+Cills
+Ciliento
+Cilibrasi
+Cilfone
+Ciesiolka
+Ciersezwski
+Cierpke
+Cierley
+Cieloha
+Cicio
+Cichosz
+Cichonski
+Cicconi
+Cibulskas
+Ciaramitaro
+Ciano
+Cianciotta
+Ciampanella
+Cialella
+Ciaccia
+Chwieroth
+Chwalek
+Chvilicek
+Chuyangher
+Churner
+Churchville
+Chuppa
+Chupik
+Chukri
+Chuh
+Chudzinski
+Chudzik
+Chudej
+Chrones
+Chroman
+Christoffer
+Christmau
+Christle
+Christaldi
+Christal
+Chrispen
+Chriscoe
+Chown
+Chowen
+Chowanec
+Chounlapane
+Choulnard
+Chott
+Chopelas
+Chomicki
+Chomali
+Choen
+Chodorov
+Chmelik
+Chludzinski
+Chivalette
+Chiv
+Chiumento
+Chittom
+Chisnall
+Chischilly
+Chisari
+Chirdon
+Chirasello
+Chipp
+Chiotti
+Chionchio
+Chioma
+Chinweze
+Chinskey
+Chinnis
+Chinni
+Chindlund
+Chimeno
+Chilinskas
+Childes
+Chikko
+Chihak
+Chiffriller
+Chieves
+Chieng
+Chiavaroli
+Chiara
+Chiapetto
+Chiaminto
+Chhor
+Chhon
+Chheng
+Chhabra
+Cheyney
+Chey
+Chevres
+Chetelat
+Chet
+Chestand
+Chessor
+Chesmore
+Chesick
+Chesanek
+Cherwinski
+Chervin
+Cherven
+Cherrie
+Chernick
+Chernay
+Cherchio
+Cheon
+Chenevey
+Chenet
+Chenauls
+Chenaille
+Chemin
+Chemell
+Chegwidden
+Cheffer
+Chefalo
+Chebret
+Chebahtah
+Cheas
+Chaven
+Chavayda
+Chautin
+Chauhdrey
+Chauffe
+Chaudet
+Chatterson
+Chatriand
+Chaton
+Chastant
+Chass
+Chasnoff
+Chars
+Charnoski
+Charleton
+Charle
+Charisse
+Charif
+Charfauros
+Chareunsri
+Chareunrath
+Charbonnel
+Chappan
+Chaples
+Chaplean
+Chapko
+Chaobal
+Chanthaumlsa
+Chantha
+Chanofsky
+Chanel
+Chandsawangbh
+Chandronnait
+Chandrasekhar
+Chandrasekara
+Chandier
+Chanchuan
+Chananie
+Chanady
+Champy
+Champany
+Chamley
+Chamers
+Chamble
+Chamberlian
+Chalow
+Chaloner
+Chalita
+Chalaban
+Chajon
+Chais
+Chaim
+Chaille
+Chaidy
+Chagollan
+Chafe
+Chadsey
+Chaderton
+Chabotte
+Cezil
+Cersey
+Cerritelli
+Ceronsky
+Ceroni
+Cernansky
+Cerenzia
+Cereghino
+Cerdan
+Cerchia
+Cerbantes
+Cerao
+Ceranski
+Centrone
+Centorino
+Censky
+Ceman
+Cely
+Celuch
+Cellupica
+Cellio
+Celani
+Cegla
+Cedars
+Ceasor
+Cearlock
+Cazzell
+Cazeault
+Caza
+Cavezon
+Cavalli
+Cavaleri
+Cavaco
+Cautillo
+Cauthorne
+Caulley
+Caughran
+Cauchon
+Catucci
+Cattladge
+Cattabriga
+Catillo
+Cathers
+Catenaccio
+Catena
+Catani
+Catalli
+Catacun
+Casumpang
+Casuat
+Castrovinci
+Castronova
+Castoral
+Castiola
+Castin
+Castillero
+Castillejo
+Castera
+Castellanoz
+Castellaneta
+Castelan
+Castanio
+Castanado
+Castagnier
+Cassis
+Cassion
+Cassello
+Casseday
+Cassase
+Cassarubias
+Cassard
+Cassaday
+Caspary
+Caspar
+Casoria
+Casilles
+Casile
+Casida
+Cashing
+Casgrove
+Caseman
+Caselton
+Casello
+Caselden
+Cascia
+Casario
+Casareno
+Casarella
+Casamayor
+Casaliggi
+Casalenda
+Casagranda
+Casabona
+Carza
+Caryk
+Carvett
+Carthew
+Carther
+Carthens
+Cartaya
+Cartan
+Carsno
+Carscallen
+Carrubba
+Carroca
+Carril
+Carrigg
+Carridine
+Carrelli
+Carraturo
+Carratura
+Carras
+Carransa
+Carrahan
+Carpente
+Carpenito
+Caroway
+Carota
+Caronna
+Caroline
+Carnoske
+Carnohan
+Carnighan
+Carnie
+Carnahiba
+Carmichel
+Carmello
+Carlsley
+Carlington
+Carleo
+Cariveau
+Caristo
+Carillion
+Carilli
+Caridine
+Cariaso
+Cardoni
+Cardish
+Cardino
+Cardinas
+Cardenos
+Cardejon
+Cardeiro
+Carco
+Carbal
+Caravalho
+Caraher
+Caradonna
+Caracso
+Caracciola
+Capshaws
+Caprice
+Capriccioso
+Capraro
+Cappaert
+Caposole
+Capitani
+Capinpin
+Capiga
+Capezzuto
+Capetl
+Capestany
+Capels
+Capellas
+Caparoula
+Caparelli
+Capalongan
+Capaldo
+Canu
+Cantre
+Cantoral
+Cantfield
+Cantabrana
+Canori
+Cannuli
+Canestro
+Canestrini
+Canerday
+Canellas
+Canella
+Candon
+Cancer
+Canatella
+Canak
+Cana
+Campolongo
+Campagnone
+Campagnini
+Campagne
+Camon
+Cammarn
+Caminita
+Camidge
+Cambronne
+Cambric
+Cambero
+Camaron
+Calzone
+Calzadilla
+Calver
+Calvent
+Calvelo
+Calvaruso
+Calvaresi
+Calpin
+Calonsag
+Calonne
+Caloca
+Calligy
+Callez
+Calleo
+Callaro
+Calixtro
+Caliguire
+Caligari
+Calicut
+Caler
+Calderson
+Caldarone
+Calchera
+Calcagino
+Calaycay
+Calamarino
+Calamari
+Calamare
+Cakanic
+Cajune
+Cajucom
+Cajero
+Cainion
+Cainglit
+Caiafa
+Cagey
+Cafourek
+Caffarel
+Cafarella
+Cafagno
+Cadoy
+Cadmen
+Cader
+Cademartori
+Cackett
+Cacibauda
+Caci
+Cacciola
+Cabrar
+Cabla
+Cabiya
+Cabido
+Cabeza
+Cabellon
+Cabeceira
+Cabanes
+Cabag
+Bzhyan
+Byther
+Byro
+Byrley
+Byrdsong
+Bynd
+Bylund
+Byant
+Bverger
+Buzzelle
+Buzzanca
+Buyes
+Buyak
+Buvens
+Buttino
+Buttimer
+Buttari
+Buttaccio
+Buther
+Butel
+Buszak
+Bustinza
+Bussom
+Busskohl
+Bussink
+Bussinger
+Bussert
+Busselberg
+Bussani
+Busl
+Buskohl
+Busie
+Bushie
+Busenius
+Buseck
+Buscarino
+Busacker
+Burwick
+Burtin
+Burriesci
+Burreson
+Burnum
+Burnet
+Burneisen
+Burnaman
+Burlette
+Burlando
+Burki
+Burker
+Burkel
+Burka
+Burigsay
+Burhanuddin
+Burgen
+Burgbacher
+Buretta
+Buress
+Burdsall
+Burdis
+Burdi
+Burdg
+Burbano
+Bur
+Buquo
+Buontempo
+Buonadonna
+Bunzey
+Bunyea
+Buntain
+Bunkers
+Bungy
+Bungart
+Bunetta
+Bunes
+Bundley
+Bundette
+Bumm
+Bumbray
+Bumba
+Bumatay
+Bulwinkle
+Bultron
+Bulnes
+Bullo
+Bullmore
+Bullerwell
+Bullert
+Bullara
+Bulland
+Bulkin
+Bulgarella
+Bulacan
+Bukrim
+Bukowinski
+Bujol
+Buja
+Buike
+Buhoveckey
+Buhite
+Bugtong
+Bugler
+Bugenhagen
+Bugayong
+Bugarewicz
+Bufton
+Buetti
+Buess
+Buerstatte
+Buergel
+Buerge
+Buer
+Buena
+Buegler
+Bueggens
+Buecher
+Budzyna
+Budz
+Budworth
+Budesa
+Buddle
+Budden
+Buddemeyer
+Buckridge
+Buckreis
+Buckmiller
+Bucke
+Buchser
+Buchsbaum
+Buchs
+Buchna
+Buchheim
+Buchberger
+Bucchin
+Bucanan
+Bubbico
+Buanno
+Bual
+Brzycki
+Brzostowski
+Bryum
+Brynga
+Brynestad
+Bryar
+Bruzewicz
+Bruyn
+Bruun
+Brutlag
+Bruson
+Bruski
+Bruse
+Brusco
+Bruscino
+Brunsting
+Brunskill
+Brunow
+Brunnemer
+Brunderman
+Brunckhorst
+Brunback
+Brumbley
+Bruh
+Brugal
+Bruenderman
+Bruegman
+Brucie
+Brozyna
+Brozell
+Brownsworth
+Brownsword
+Brownsberger
+Browley
+Brous
+Brounson
+Broumley
+Brostoff
+Brossmann
+Brosig
+Broschinsky
+Broomell
+Brookshier
+Brooklyn
+Bronikowski
+Brondyke
+Bromberek
+Brombach
+Brokins
+Broking
+Brojakowski
+Broich
+Brogren
+Brogglin
+Brodhurst
+Brodhag
+Brodey
+Brocklebank
+Brockie
+Brockell
+Brochure
+Brochhausen
+Broccolo
+Brixius
+Brittsan
+Brits
+Britnell
+Brisley
+Brisbone
+Briola
+Brintnall
+Bringman
+Bringas
+Bringantino
+Brinckerhoff
+Briguglio
+Briggerman
+Brigg
+Brigantino
+Briehl
+Brieger
+Bridson
+Bridjmohan
+Bridgford
+Bridget
+Bridgens
+Bridendolph
+Briden
+Briddick
+Bricknell
+Brickles
+Brichetto
+Briare
+Brez
+Brevitz
+Brevil
+Breutzmann
+Breuning
+Bretl
+Brethour
+Bretana
+Bresolin
+Breslawski
+Brentnall
+Brentano
+Brensnan
+Brensinger
+Brensel
+Brenowitz
+Brennenstuhl
+Brengle
+Brendlinger
+Brenda
+Brend
+Brence
+Brenaman
+Bremseth
+Bremme
+Breman
+Brelje
+Breitung
+Breitenfeldt
+Breitenbucher
+Breitenberg
+Breines
+Breiland
+Brehony
+Bregon
+Brege
+Bregantini
+Brefka
+Breeman
+Breehl
+Bredy
+Bredow
+Bredice
+Bredahl
+Brechbill
+Brearley
+Brdar
+Brazzi
+Brazler
+Braye
+Braver
+Bravender
+Bravard
+Braunsdorf
+Braunschweige
+Braught
+Brauchla
+Bratek
+Braskey
+Brasket
+Branske
+Branot
+Branine
+Braniff
+Brangan
+Branen
+Branecki
+Brandsrud
+Brandman
+Brandeland
+Brande
+Brandauer
+Brancazio
+Brancanto
+Branaugh
+Bramucci
+Brakstad
+Brais
+Braim
+Braig
+Brah
+Brage
+Bradtke
+Bradrick
+Bradon
+Bradicich
+Brackelsberg
+Brachman
+Brachle
+Bracetty
+Bracaloni
+Bozzell
+Bozovich
+Bozinovich
+Boyenga
+Bowring
+Bowlet
+Bowgren
+Bowersmith
+Bowels
+Bowcutt
+Bovio
+Boveja
+Bovain
+Boutchyard
+Bousson
+Bousqute
+Bousley
+Bourns
+Bourlier
+Bourgois
+Bourff
+Bourek
+Bourdeaux
+Bourdages
+Bourbonnais
+Boundy
+Bouliouris
+Boudrieau
+Boudin
+Bouchaert
+Botwin
+Bottomly
+Bottolfson
+Bottolene
+Bottiggi
+Botterbusch
+Botros
+Botras
+Botdorf
+Bostelman
+Bossenbroek
+Bossardet
+Bosowski
+Boschult
+Borycz
+Borwig
+Boruvka
+Bortignon
+Borsa
+Borromeo
+Borrolli
+Borries
+Borreta
+Borremans
+Borras
+Borr
+Borozny
+Borowiec
+Boronat
+Bornman
+Bormes
+Borlin
+Borguez
+Borgstede
+Borgese
+Borgert
+Borgers
+Borgella
+Borell
+Bordon
+Bordi
+Bordges
+Bordenkircher
+Borde
+Borbon
+Boratko
+Boque
+Boppre
+Boosalis
+Boorom
+Bookter
+Bookmiller
+Bookamer
+Bonzo
+Bonyai
+Bonugli
+Bonsu
+Bonsey
+Bonsell
+Bonsee
+Bonow
+Bonno
+Bonnlander
+Bonnin
+Bonnenfant
+Bonjorno
+Boniol
+Bongo
+Bonetto
+Bonepart
+Bondre
+Bonaventura
+Bonatti
+Bonapart
+Bonagurio
+Bonaguidi
+Bomzer
+Bompane
+Bomilla
+Bomia
+Bombino
+Bomaster
+Bollens
+Bollbach
+Bollaert
+Bolins
+Bolinder
+Bolig
+Bolian
+Bolfa
+Bolevice
+Boldwyn
+Bolduan
+Boldizsar
+Bolde
+Bokal
+Boitel
+Boin
+Boillot
+Boid
+Bohonik
+Bohnker
+Bohney
+Bohlsen
+Bohlman
+Bohlken
+Bogut
+Bognuda
+Bogguess
+Bogg
+Bofinger
+Boero
+Boerm
+Boeri
+Boera
+Boelk
+Boehnke
+Boege
+Bodyfelt
+Bodon
+Bodison
+Bodfish
+Boderick
+Bodenhagen
+Bodelson
+Bodary
+Bocskor
+Bockrath
+Bocklund
+Bockhorn
+Bockenstedt
+Bockelmann
+Bochicchio
+Boches
+Bochek
+Bocchieri
+Boccard
+Bobsin
+Bobrosky
+Bobowiec
+Boblak
+Bobet
+Boane
+Boamah
+Blyze
+Blute
+Blush
+Blunkall
+Blundo
+Blumkin
+Bluming
+Blumenschein
+Blumenkrantz
+Blumenberg
+Bluel
+Bloye
+Blott
+Blotsky
+Blossomgame
+Blosfield
+Bloomstrom
+Bloomstrand
+Bloomsburg
+Blonsky
+Blonigan
+Blomstrand
+Bloes
+Bloemker
+Bloedel
+Blochberger
+Blizard
+Blinebry
+Blindt
+Blihovde
+Blide
+Blicker
+Bleything
+Blevans
+Blessett
+Blesofsky
+Bleiler
+Bleichner
+Bleicher
+Bleeck
+Blee
+Blazon
+Blazing
+Blazich
+Blaydon
+Blaxland
+Blauw
+Blauman
+Blaszczyk
+Blasl
+Blashak
+Blasenhauer
+Blanscet
+Blanquet
+Blanquart
+Blannon
+Blanko
+Blankenbecler
+Blanga
+Blander
+Blakstad
+Blailock
+Blafield
+Blaeser
+Blaese
+Blady
+Bladt
+Blacock
+Blackwall
+Blackmoore
+Blackmar
+Blackington
+Blackbird
+Blacio
+Blachowski
+Bjornstrom
+Bjorn
+Bjerknes
+Bjerken
+Bjella
+Bizzard
+Bivans
+Bitzenhofer
+Bitar
+Bitah
+Bissol
+Bissel
+Bissada
+Bispham
+Bisikirski
+Bischel
+Biscari
+Bisanz
+Birthwright
+Birsner
+Bironas
+Birner
+Birnberg
+Birkmaier
+Birkenhagen
+Birely
+Birdon
+Bionda
+Binn
+Bininger
+Binet
+Binderup
+Binam
+Billus
+Billue
+Billotti
+Billinsley
+Billingsby
+Billigmeier
+Billiet
+Billiar
+Billesbach
+Bilchak
+Bilansky
+Bijan
+Bihler
+Bihl
+Bigusiak
+Bigony
+Bignell
+Biggard
+Biewald
+Biever
+Bietsch
+Biesenthal
+Biesecker
+Bierut
+Bierstedt
+Bierschbach
+Biersack
+Bierod
+Bierl
+Bierkortte
+Biener
+Bielser
+Bielke
+Bielefield
+Biedekapp
+Bidstrup
+Bidell
+Biddlecome
+Bicknase
+Bicking
+Bichoupan
+Bichoff
+Bibiloni
+Biastock
+Biasotti
+Bianchin
+Bhullar
+Bhaskar
+Bhamaraniyama
+Bhairo
+Bezenek
+Beyser
+Beyke
+Beyea
+Beydoun
+Beyale
+Beyal
+Bevevino
+Beuttel
+Beutnagel
+Beuthin
+Beuse
+Beurskens
+Beukema
+Beukelman
+Beuerle
+Beuchler
+Betzner
+Betzler
+Betzig
+Bettley
+Betry
+Betit
+Bethurem
+Betha
+Betenson
+Betak
+Bestwick
+Bestine
+Beste
+Bessone
+Bessinger
+Bessellieu
+Besong
+Besner
+Beskom
+Beshore
+Beser
+Besen
+Beseke
+Besares
+Besant
+Besanson
+Besancon
+Berzunza
+Berulie
+Bertrum
+Bertot
+Berto
+Bertman
+Berther
+Berth
+Bertella
+Bertao
+Bershadsky
+Bersaw
+Berrospe
+Berrocal
+Berray
+Bernstock
+Bernotas
+Bernos
+Bernmen
+Bernitsky
+Bernieri
+Berni
+Bernheim
+Berneri
+Bernell
+Bernbeck
+Bernaudo
+Bernau
+Bernatchez
+Bernarducci
+Bernardon
+Bernand
+Bernacki
+Berlingo
+Berley
+Berlandy
+Berlacher
+Berkovitch
+Berkenbile
+Berkbigler
+Berishaj
+Bering
+Bergstedt
+Bergsman
+Bergouignan
+Bergold
+Bergmeyer
+Bergfalk
+Bergenty
+Bergenstock
+Bergene
+Bergamine
+Bergami
+Berey
+Beresik
+Berentz
+Berenschot
+Bereda
+Berdux
+Berdar
+Berdahl
+Berczy
+Berchielli
+Bercher
+Berceir
+Berbig
+Berbereia
+Benzee
+Benwarc
+Benulis
+Bentzinger
+Bentrem
+Benthusen
+Benston
+Bennings
+Bennight
+Benneth
+Bennard
+Bennafield
+Benkosky
+Benker
+Benje
+Benisek
+Benintendi
+Bening
+Beninati
+Benimadho
+Benezra
+Beneuento
+Bendu
+Bending
+Bendell
+Benckendorf
+Benbenek
+Benanti
+Benamati
+Benafield
+Benach
+Benac
+Bembi
+Belwood
+Belvees
+Beltramo
+Belstad
+Belski
+Belschner
+Belscher
+Belovs
+Belousson
+Belous
+Belony
+Belonger
+Belluz
+Bellmore
+Bellitti
+Belliston
+Bellingtier
+Bellinder
+Bellhouse
+Bellflowers
+Bellen
+Bellehumeur
+Bellefontaine
+Bellar
+Bellantone
+Bellair
+Bellace
+Belken
+Belke
+Beliz
+Belina
+Belieu
+Belidor
+Beliard
+Belhumeur
+Belfy
+Belfort
+Belfi
+Belfast
+Belezos
+Belchior
+Belarmino
+Belanich
+Belancer
+Bejil
+Bejger
+Bejerano
+Beja
+Beiswenger
+Beissel
+Beilstein
+Beilinson
+Beilfuss
+Beile
+Behner
+Behizadeh
+Behimer
+Beherns
+Behanan
+Behal
+Begun
+Beguhl
+Begonia
+Begolli
+Begnoche
+Begen
+Beese
+Beerle
+Beemon
+Beelar
+Beedoo
+Beedles
+Beedham
+Beeckman
+Beebout
+Bedre
+Bedocs
+Bednarowicz
+Bedlion
+Bedillion
+Beder
+Bedenfield
+Bedee
+Bedaw
+Bedatsky
+Bedar
+Beckor
+Becklin
+Beckes
+Beckelheimer
+Beaureguard
+Beauparlant
+Beau
+Beattle
+Beatson
+Beath
+Beards
+Bearded
+Beandoin
+Beady
+Beachman
+Beachell
+Bayus
+Baysden
+Bayouth
+Bayon
+Bayn
+Bayani
+Baxtor
+Bawks
+Bawer
+Bawcombe
+Baves
+Bautiste
+Baute
+Baurer
+Baumohl
+Baumli
+Baumkirchner
+Baumiester
+Baumgartel
+Baumgarn
+Baumfalk
+Bauchspies
+Bauce
+Batzri
+Battisto
+Batter
+Battenhouse
+Batteiger
+Batrich
+Batra
+Batlle
+Batlis
+Batliner
+Batkin
+Batchellor
+Bastick
+Bastardi
+Bassiti
+Basore
+Basone
+Baskow
+Basini
+Basila
+Bashline
+Baseley
+Bascas
+Barvosa
+Barvick
+Barus
+Bartuska
+Bartula
+Bartosik
+Bartosch
+Bartoli
+Bartmes
+Bartlette
+Bartkus
+Bartkiewicz
+Bartholomeu
+Barte
+Bartch
+Barsegyan
+Barschdoor
+Barscewski
+Barsamian
+Barryman
+Barrowman
+Barrois
+Barrish
+Barriault
+Barrete
+Barree
+Barran
+Baronne
+Barninger
+Barners
+Barnebey
+Barnak
+Barnacle
+Barlup
+Barlock
+Barlau
+Barlak
+Barken
+Barkema
+Barjenbruch
+Barillo
+Barill
+Barientos
+Baria
+Bargstadt
+Bargmann
+Bargeron
+Baresi
+Barera
+Barends
+Bardos
+Bardoner
+Bardill
+Bardell
+Barck
+Barcik
+Barchus
+Barchacky
+Barberr
+Barbaza
+Barbarito
+Barbare
+Barbalich
+Barbadillo
+Baranga
+Barahana
+Baradi
+Barad
+Barach
+Barabin
+Baquero
+Banwarth
+Bansmer
+Banse
+Banowski
+Bannett
+Bankos
+Bangura
+Banerji
+Banek
+Bandyk
+Bandura
+Bandasak
+Bandarra
+Bancourt
+Banco
+Bancks
+Banbury
+Bamforth
+Bambas
+Bambace
+Balzotti
+Balzarine
+Balza
+Balwinski
+Baltruweit
+Baltazor
+Balsis
+Baloy
+Balow
+Balock
+Balo
+Balm
+Balluch
+Ballowe
+Ballmann
+Ballez
+Balletto
+Ballesterous
+Ballena
+Ballejos
+Ballar
+Ballan
+Ballagas
+Balitas
+Balish
+Baligod
+Balich
+Baldwyn
+Balduzzi
+Baldos
+Balderree
+Baldearena
+Balda
+Balcos
+Balasko
+Balangatan
+Balak
+Baladejo
+Bakalars
+Bajko
+Bajek
+Baitner
+Baison
+Bairo
+Baiotto
+Bainey
+Bailleu
+Bailado
+Baibak
+Bahri
+Bahde
+Bahadue
+Bagwill
+Bagu
+Bagron
+Bagnaschi
+Baffa
+Baff
+Baeskens
+Baerg
+Baenziger
+Baena
+Baell
+Badzinski
+Badruddin
+Badlam
+Badey
+Badertscher
+Badenoch
+Badagliacca
+Bacone
+Bacman
+Backhuus
+Bacino
+Bachmeyer
+Bachinski
+Bachas
+Bachan
+Bacerra
+Bacayo
+Babson
+Bablak
+Babinski
+Babilon
+Babikian
+Babicz
+Babey
+Babbish
+Baarts
+Baack
+Azznara
+Azuma
+Azor
+Azatyan
+Azapinto
+Azahar
+Ayyad
+Aytes
+Aysien
+Aymar
+Aylock
+Ayhens
+Ayele
+Aydin
+Axtman
+Axman
+Awyie
+Aw
+Avona
+Avner
+Avison
+Avenia
+Aveles
+Avarbuch
+Avancena
+Autullo
+Autovino
+Autobee
+Auther
+Auter
+Austino
+Austine
+Auster
+Auslam
+Aurrichio
+Aun
+Auls
+Aulder
+Aufiero
+Audrey
+Audibert
+Audelhuk
+Auckley
+Auces
+Aubel
+Auala
+Atzinger
+Atzhorn
+Attwell
+Attles
+Attilio
+Attia
+Atthowe
+Atteburg
+Atmore
+Atma
+Atleh
+Atkisson
+Athy
+Atherholt
+Athanasiou
+Atengco
+Atamanczyk
+Astillero
+Astafan
+Assum
+Assis
+Assing
+Assenmacher
+Assalone
+Assael
+Asrari
+Aspri
+Aspley
+Asperheim
+Aspell
+Asnicar
+Asner
+Askiew
+Askia
+Aske
+Ask
+Ashly
+Ashkettle
+Ashing
+Ashbourne
+Ashbach
+Ashaf
+Asenjo
+Aseng
+Aseltine
+Ascol
+Aschbacher
+Asamoah
+Arzt
+Arzabala
+Arview
+Arvez
+Arvanitis
+Arva
+Arunachalam
+Arton
+Arties
+Artibee
+Arthun
+Artez
+Arters
+Arsham
+Arseneault
+Arroyd
+Arroyano
+Arrospide
+Arrocho
+Arrisola
+Arrindel
+Arrigone
+Arrellin
+Arredla
+Arrand
+Arrance
+Arquelles
+Arosemena
+Arollo
+Aroca
+Arntzen
+Arnsberger
+Arnitz
+Arnerich
+Arndell
+Arnaudet
+Arnao
+Arnaldo
+Army
+Armout
+Armold
+Armocida
+Armlin
+Armiso
+Armesto
+Armen
+Armada
+Arkontaky
+Arking
+Aristizabal
+Arisa
+Arildsen
+Arichabala
+Ariail
+Argulewicz
+Argudin
+Argro
+Argie
+Argenziano
+Argenti
+Arendash
+Arendall
+Arendale
+Arelleano
+Arehano
+Ards
+Ardeneaux
+Ardelean
+Ardaly
+Arciola
+Arcieri
+Archiopoli
+Archdale
+Archbell
+Arbon
+Arbolida
+Arbetman
+Arbertha
+Arau
+Arashiro
+Araneo
+Arancibia
+Araldi
+Aragones
+Aragao
+Arabajian
+Aquas
+Apthorpe
+Apshire
+Aprill
+Aprigliano
+Applonie
+Appl
+Appia
+Appana
+Aponta
+Aplington
+Apley
+Apker
+Apelian
+Apadaca
+Aono
+Ao
+Anzideo
+Anway
+Antronica
+Antosh
+Antonovich
+Antoniak
+Antolak
+Antila
+Antignani
+Anthes
+Antao
+Ansoategui
+Ansloan
+Anreozzi
+Anos
+Anolick
+Anoe
+Annuzzi
+Anning
+Annarino
+Annal
+Annable
+Annabel
+Anitok
+Aninion
+Animashaun
+Anidi
+Angocicco
+Angland
+Angiolelli
+Angileri
+Angilello
+Angier
+Angermeier
+Angelozzi
+Angelou
+Angellotti
+Angelillo
+Angelica
+Angalich
+Aney
+Anewalt
+Anetsberger
+Anesi
+Aneshansley
+Anene
+Anecelle
+Andrzejczyk
+Andrzejczak
+Andruszkiewic
+Andrson
+Androde
+Andriopulos
+Andrino
+Andrich
+Andreola
+Andregg
+Andreessen
+Andrango
+Andradez
+Andrades
+Andrachak
+Andoh
+Andina
+Anderst
+Anderholm
+Andere
+Andalora
+Anciso
+Ancic
+Ancel
+Ancar
+Ancalade
+Anawaty
+Anawalt
+Amys
+Amstrong
+Amspaugh
+Amous
+Amott
+Amoros
+Amormino
+Amoriello
+Amorello
+Amoe
+Amodt
+Ammonds
+Ammirata
+Ammer
+Amlin
+Amith
+Amistadi
+Amill
+Amigo
+Amerio
+American
+Amentler
+Amemiya
+Amela
+Amejorado
+Amedro
+Amedeo
+Amburgy
+Ambroziak
+Ambrister
+Amboree
+Amboise
+Ambert
+Ambagis
+Amauty
+Amat
+Amas
+Amarian
+Amara
+Amalong
+Alwin
+Alwazan
+Alvirez
+Alvero
+Alverado
+Alty
+Altstatt
+Altsisi
+Altmark
+Altimus
+Altamiruno
+Alson
+Alsing
+Alsaqri
+Alrod
+Alquesta
+Alpis
+Alpheaus
+Alperin
+Aloy
+Alosta
+Aloan
+Alnoor
+Almsteadt
+Almstead
+Almos
+Almgren
+Almarza
+Almajhoub
+Allyne
+Allsbrooks
+Allon
+Allinger
+Alliman
+Alliance
+Allgire
+Allevato
+Alleshouse
+Alleruzzo
+Allerton
+Allder
+Allcock
+Allbert
+Allanson
+Allabaugh
+Alkins
+Alkema
+Alkana
+Aljemal
+Alisauskas
+Alimo
+Alimento
+Alie
+Alicer
+Alias
+Alhusseini
+Alhameed
+Alhambra
+Alhaddad
+Alfredo
+Alfiero
+Aleyandrez
+Alexidor
+Alexandropoul
+Alexanders
+Alexakis
+Alesse
+Alesna
+Alepin
+Alejandrez
+Aldworth
+Aldrow
+Aldrige
+Aldonza
+Alcine
+Alcantas
+Albu
+Albrough
+Albor
+Albe
+Albarracin
+Albarazi
+Alatosse
+Alarcone
+Alanko
+Aland
+Alamia
+Alameida
+Alambar
+Alai
+Akwei
+Aksoy
+Ako
+Akley
+Akinrefon
+Akimseu
+Akhavan
+Akhand
+Akery
+Akawanzie
+Akapo
+Akamiro
+Akal
+Ajoku
+Ajani
+Aiuto
+Aiudi
+Airth
+Aipperspach
+Aiporlani
+Aipopo
+Aiola
+Aini
+Ailsworth
+Aills
+Ailiff
+Aievoli
+Aid
+Aiava
+Ahyet
+Ahrenholz
+Ahnell
+Ahlo
+Ahlfield
+Ahlemeyer
+Ahimud
+Ahia
+Ahhee
+Ahaus
+Ahalt
+Agustino
+Agustine
+Agurs
+Agumga
+Aguele
+Agresto
+Agreda
+Agpaoa
+Agosti
+Agoro
+Agonoy
+Agoff
+Aggers
+Agemy
+Ageboi
+Agbisit
+Afurong
+Afshar
+Affronti
+Afflick
+Affeltranger
+Afable
+Aeillo
+Adule
+Adrion
+Adolphe
+Adolfson
+Adner
+Adloff
+Adling
+Adickes
+Adib
+Adelsperger
+Adelmund
+Adelizzi
+Addeo
+Adamsonis
+Adamsen
+Adamowski
+Adamos
+Adamec
+Adalja
+Acosto
+Acors
+Acorda
+Acock
+Acly
+Ackah
+Achin
+Aceveda
+Acerra
+Acerno
+Aceituno
+Acee
+Accala
+Acal
+Abusufait
+Abugn
+Abuel
+Absalon
+Abriola
+Abrey
+Abrell
+Abramovitz
+Abramoff
+Abramian
+Abrahamian
+Abousaleh
+Aboshihata
+Abolafia
+Ableman
+Abkemeier
+Abington
+Abina
+Abigantus
+Abide
+Abeta
+Abercombie
+Abdulmuniem
+Abdulaziz
+Abdou
+Abdelmuti
+Abdelaziz
+Abdelal
+Abbington
+Abbatiello
+Abajian
+Abaja
+Aarsvold
+Aarhus
+Aardema
+Aarant
+Aanderud
+Aalund
+Aalderink
diff --git a/hyracks/hyracks-storage-am-invertedindex/pom.xml b/hyracks/hyracks-storage-am-invertedindex/pom.xml
deleted file mode 100644
index 5fe2d96..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/pom.xml
+++ /dev/null
@@ -1,65 +0,0 @@
-<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>hyracks-storage-am-invertedindex</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
-  <name>hyracks-storage-am-invertedindex</name>
-
-  <parent>
-    <groupId>edu.uci.ics.hyracks</groupId>
-    <artifactId>hyracks</artifactId>
-    <version>0.2.3-SNAPSHOT</version>
-  </parent>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <version>2.0.2</version>
-        <configuration>
-          <source>1.7</source>
-          <target>1.7</target>
-          <fork>true</fork>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-  <dependencies>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-storage-common</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>  	
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-dataflow-common</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>  	
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>  	
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-storage-am-btree</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>    
-  	<dependency>
-  		<groupId>junit</groupId>
-  		<artifactId>junit</artifactId>
-  		<version>4.8.1</version>
-  		<type>jar</type>
-  		<scope>test</scope>
-  	</dependency>  	  		
-  </dependencies>
-</project>
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
deleted file mode 100644
index 042042f..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.api;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-
-public interface IInvertedIndexOperatorDescriptor extends ITreeIndexOperatorDescriptor {
-    public IBinaryComparatorFactory[] getInvListsComparatorFactories();
-
-    public IBinaryTokenizerFactory getTokenizerFactory();
-    
-    public ITypeTraits[] getInvListsTypeTraits();
-    
-    public IFileSplitProvider getInvListsFileSplitProvider();
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifier.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifier.java
deleted file mode 100644
index bd96f67..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifier.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.api;
-
-import java.util.List;
-
-public interface IInvertedIndexSearchModifier {
-    public int getOccurrenceThreshold(List<IInvertedListCursor> invListCursors);
-
-    public int getPrefixLists(List<IInvertedListCursor> invListCursors);
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifierFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifierFactory.java
deleted file mode 100644
index 92770d6..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifierFactory.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.api;
-
-import java.io.Serializable;
-
-public interface IInvertedIndexSearchModifierFactory extends Serializable {
-    public IInvertedIndexSearchModifier createSearchModifier();
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearcher.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearcher.java
deleted file mode 100644
index d633c34..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearcher.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.api;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndexSearchCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndexSearchPredicate;
-
-public interface IInvertedIndexSearcher {
-    public void search(InvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred)
-            throws HyracksDataException, IndexException;
-
-    public IFrameTupleAccessor createResultFrameTupleAccessor();
-
-    public ITupleReference createResultTupleReference();
-
-    public List<ByteBuffer> getResultBuffers();
-
-    public int getNumValidResultBuffers();
-    
-    public void reset();
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedListBuilder.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedListBuilder.java
deleted file mode 100644
index aaaef56..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedListBuilder.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.api;
-
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-
-public interface IInvertedListBuilder {
-    public boolean startNewList(ITupleReference tuple, int numTokenFields);
-
-    // returns true if successfully appended
-    // returns false if not enough space in targetBuf
-    public boolean appendElement(ITupleReference tuple, int numTokenFields, int numElementFields);
-
-    public void setTargetBuffer(byte[] targetBuf, int startPos);
-
-    public int getListSize();
-
-    public int getPos();
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedListCursor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedListCursor.java
deleted file mode 100644
index 9435f3c..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedListCursor.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.api;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-
-public interface IInvertedListCursor extends Comparable<IInvertedListCursor> {
-    void reset(int startPageId, int endPageId, int startOff, int numElements);
-
-    void pinPagesSync() throws HyracksDataException;
-
-    void pinPagesAsync() throws HyracksDataException;
-
-    void unpinPages() throws HyracksDataException;
-
-    boolean hasNext();
-
-    void next();
-
-    ITupleReference getTuple();
-
-    // getters
-    int getNumElements();
-
-    int getStartPageId();
-
-    int getEndPageId();
-
-    int getStartOff();
-
-    // jump to a specific element
-    void positionCursor(int elementIx);
-
-    boolean containsKey(ITupleReference searchTuple, MultiComparator invListCmp);
-
-    // for debugging
-    String printInvList(ISerializerDeserializer[] serdes) throws HyracksDataException;
-
-    String printCurrentElement(ISerializerDeserializer[] serdes) throws HyracksDataException;
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
deleted file mode 100644
index 117f96f..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-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.IOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.util.InvertedIndexUtils;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public abstract class AbstractInvertedIndexOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor
-        implements IInvertedIndexOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    // General.
-    protected final IStorageManagerInterface storageManager;
-    protected final IIndexRegistryProvider<IIndex> indexRegistryProvider;
-    protected final boolean retainInput;
-    protected final IOperationCallbackProvider opCallbackProvider;
-    
-    // Btree.
-    protected final ITreeIndexFrameFactory btreeInteriorFrameFactory;
-    protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
-    protected final ITypeTraits[] btreeTypeTraits;
-    protected final IBinaryComparatorFactory[] btreeComparatorFactories;
-    protected final IIndexDataflowHelperFactory btreeDataflowHelperFactory;
-    protected final IFileSplitProvider btreeFileSplitProvider;
-
-    // Inverted index.
-    protected final ITypeTraits[] invListsTypeTraits;
-    protected final IBinaryComparatorFactory[] invListComparatorFactories;
-    protected final IBinaryTokenizerFactory tokenizerFactory;
-    protected final IFileSplitProvider invListsFileSplitProvider;
-
-    public AbstractInvertedIndexOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity, int outputArity,
-            RecordDescriptor recDesc, IStorageManagerInterface storageManager,
-            IFileSplitProvider btreeFileSplitProvider, IFileSplitProvider invListsFileSplitProvider,
-            IIndexRegistryProvider<IIndex> indexRegistryProvider, ITypeTraits[] tokenTypeTraits,
-            IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
-            IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
-            IIndexDataflowHelperFactory btreeDataflowHelperFactory, boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
-        super(spec, inputArity, outputArity);
-
-        // General.
-        this.storageManager = storageManager;
-        this.indexRegistryProvider = indexRegistryProvider;
-        this.retainInput = retainInput;
-        this.opCallbackProvider = opCallbackProvider;
-        
-        // Btree.
-        this.btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
-        ITreeIndexTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(btreeTypeTraits);
-        this.btreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
-        this.btreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
-        this.btreeComparatorFactories = tokenComparatorFactories;
-        this.btreeDataflowHelperFactory = btreeDataflowHelperFactory;
-        this.btreeFileSplitProvider = btreeFileSplitProvider;
-
-        // Inverted index.
-        this.invListsTypeTraits = invListsTypeTraits;
-        this.invListComparatorFactories = invListComparatorFactories;
-        this.tokenizerFactory = tokenizerFactory;
-        this.invListsFileSplitProvider = invListsFileSplitProvider;
-
-        if (outputArity > 0) {
-            recordDescriptors[0] = recDesc;
-        }
-    }
-
-    @Override
-    public IFileSplitProvider getFileSplitProvider() {
-        return btreeFileSplitProvider;
-    }
-    
-    @Override
-    public IFileSplitProvider getInvListsFileSplitProvider() {
-        return invListsFileSplitProvider;
-    }
-
-    @Override
-    public IBinaryComparatorFactory[] getTreeIndexComparatorFactories() {
-        return btreeComparatorFactories;
-    }
-
-    @Override
-    public ITypeTraits[] getTreeIndexTypeTraits() {
-        return btreeTypeTraits;
-    }
-
-    @Override
-    public IStorageManagerInterface getStorageManager() {
-        return storageManager;
-    }
-
-    @Override
-    public RecordDescriptor getRecordDescriptor() {
-        return recordDescriptors[0];
-    }
-
-    @Override
-    public IBinaryComparatorFactory[] getInvListsComparatorFactories() {
-        return invListComparatorFactories;
-    }
-
-    @Override
-    public IBinaryTokenizerFactory getTokenizerFactory() {
-        return tokenizerFactory;
-    }
-    
-    @Override
-    public ITypeTraits[] getInvListsTypeTraits() {
-        return invListsTypeTraits;
-    }
-
-    @Override
-    public IIndexRegistryProvider<IIndex> getIndexRegistryProvider() {
-        return indexRegistryProvider;
-    }
-    
-    @Override
-    public IIndexDataflowHelperFactory getIndexDataflowHelperFactory() {
-        return btreeDataflowHelperFactory;
-    }
-    
-    @Override
-    public boolean getRetainInput() {
-    	return retainInput;
-    }
-    
-    @Override
-    public IOperationCallbackProvider getOpCallbackProvider() {
-    	return opCallbackProvider;
-    }
-    
-    @Override
-	public ITupleFilterFactory getTupleFilterFactory() {
-		return null;
-	}
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
deleted file mode 100644
index a146479..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-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;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-
-public class BinaryTokenizerOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    private final IBinaryTokenizerFactory tokenizerFactory;
-    // Fields that will be tokenized
-    private final int[] tokenFields;
-    // operator will append these key fields to each token, e.g., as
-    // payload for an inverted list
-    // WARNING: too many key fields can cause significant data blowup.
-    private final int[] keyFields;
-
-    public BinaryTokenizerOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IBinaryTokenizerFactory tokenizerFactory, int[] tokenFields, int[] keyFields) {
-        super(spec, 1, 1);
-        this.tokenizerFactory = tokenizerFactory;
-        this.tokenFields = tokenFields;
-        this.keyFields = keyFields;
-        recordDescriptors[0] = recDesc;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        return new BinaryTokenizerOperatorNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(
-                getActivityId(), 0), recordDescriptors[0], tokenizerFactory.createTokenizer(), tokenFields, keyFields);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
deleted file mode 100644
index 6744f70..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-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.data.std.util.GrowableArray;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-
-public class BinaryTokenizerOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-
-    private final IHyracksTaskContext ctx;
-    private final IBinaryTokenizer tokenizer;
-    private final int[] tokenFields;
-    private final int[] projFields;
-    private final RecordDescriptor inputRecDesc;
-    private final RecordDescriptor outputRecDesc;
-
-    private FrameTupleAccessor accessor;
-    private ArrayTupleBuilder builder;
-    private GrowableArray builderFieldData;
-    private FrameTupleAppender appender;
-    private ByteBuffer writeBuffer;
-
-    public BinaryTokenizerOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc,
-            RecordDescriptor outputRecDesc, IBinaryTokenizer tokenizer, int[] tokenFields, int[] projFields) {
-        this.ctx = ctx;
-        this.tokenizer = tokenizer;
-        this.tokenFields = tokenFields;
-        this.projFields = projFields;
-        this.inputRecDesc = inputRecDesc;
-        this.outputRecDesc = outputRecDesc;
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
-        writeBuffer = ctx.allocateFrame();
-        builder = new ArrayTupleBuilder(outputRecDesc.getFieldCount());
-        builderFieldData = builder.getFieldData();
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        appender.reset(writeBuffer, true);
-        writer.open();
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        accessor.reset(buffer);
-
-        int tupleCount = accessor.getTupleCount();
-        for (int i = 0; i < tupleCount; i++) {
-
-            for (int j = 0; j < tokenFields.length; j++) {
-
-                tokenizer.reset(
-                        accessor.getBuffer().array(),
-                        accessor.getTupleStartOffset(i) + accessor.getFieldSlotsLength()
-                                + accessor.getFieldStartOffset(i, tokenFields[j]),
-                        accessor.getFieldLength(i, tokenFields[j]));
-
-                while (tokenizer.hasNext()) {
-                    tokenizer.next();
-
-                    builder.reset();
-                    try {
-                        IToken token = tokenizer.getToken();
-                        token.serializeToken(builderFieldData);
-                        builder.addFieldEndOffset();
-                    } catch (IOException e) {
-                        throw new HyracksDataException(e.getMessage());
-                    }
-
-                    for (int k = 0; k < projFields.length; k++) {
-                        builder.addField(accessor, i, projFields[k]);
-                    }
-
-                    if (!appender.append(builder.getFieldEndOffsets(), builder.getByteArray(), 0, builder.getSize())) {
-                        FrameUtils.flushFrame(writeBuffer, writer);
-                        appender.reset(writeBuffer, true);
-                        if (!appender
-                                .append(builder.getFieldEndOffsets(), builder.getByteArray(), 0, builder.getSize())) {
-                            throw new IllegalStateException();
-                        }
-                    }
-                }
-            }
-        }
-
-        if (appender.getTupleCount() > 0) {
-            FrameUtils.flushFrame(writeBuffer, writer);
-        }
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        writer.close();
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        writer.fail();
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
deleted file mode 100644
index 7ffec4d..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public class InvertedIndexBulkLoadOperatorDescriptor extends AbstractInvertedIndexOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    private final int[] fieldPermutation;
-
-    public InvertedIndexBulkLoadOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] fieldPermutation,
-            IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
-            IFileSplitProvider invListsFileSplitProvider, IIndexRegistryProvider<IIndex> indexRegistryProvider,
-            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
-            ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
-            IBinaryTokenizerFactory tokenizerFactory, IIndexDataflowHelperFactory btreeDataflowHelperFactory,
-            IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
-                indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
-                invListComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory, false,
-                opCallbackProvider);
-        this.fieldPermutation = fieldPermutation;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new InvertedIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, recordDescProvider);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
deleted file mode 100644
index c6fa56d..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-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;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex.InvertedIndexBulkLoadContext;
-
-public class InvertedIndexBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
-    private final TreeIndexDataflowHelper btreeDataflowHelper;
-    private final InvertedIndexDataflowHelper invIndexDataflowHelper;
-    private InvertedIndex invIndex;
-    private InvertedIndex.InvertedIndexBulkLoadContext bulkLoadCtx;
-
-    private FrameTupleAccessor accessor;
-    private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
-
-    private IRecordDescriptorProvider recordDescProvider;
-
-    public InvertedIndexBulkLoadOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
-            IHyracksTaskContext ctx, int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider) {
-        btreeDataflowHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory()
-                .createIndexDataflowHelper(opDesc, ctx, partition);
-        invIndexDataflowHelper = new InvertedIndexDataflowHelper(btreeDataflowHelper, opDesc, ctx, partition);
-        this.recordDescProvider = recordDescProvider;
-        tuple.setFieldPermutation(fieldPermutation);
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) btreeDataflowHelper
-                .getOperatorDescriptor();
-        RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(btreeDataflowHelper.getHyracksTaskContext().getFrameSize(), recDesc);
-
-        // BTree.
-        try {
-            btreeDataflowHelper.init(false);
-        } catch (Exception e) {
-            // Cleanup in case of failure.
-            btreeDataflowHelper.deinit();
-            if (e instanceof HyracksDataException) {
-                throw (HyracksDataException) e;
-            } else {
-                throw new HyracksDataException(e);
-            }
-        }
-
-        // Inverted Index.
-        try {
-            invIndexDataflowHelper.init(false);
-            invIndex = (InvertedIndex) invIndexDataflowHelper.getIndex();
-            bulkLoadCtx = (InvertedIndexBulkLoadContext) invIndex.beginBulkLoad(BTree.DEFAULT_FILL_FACTOR);
-        } catch (Exception e) {
-            // Cleanup in case of failure.
-            invIndexDataflowHelper.deinit();
-            if (e instanceof HyracksDataException) {
-                throw (HyracksDataException) e;
-            } else {
-                throw new HyracksDataException(e);
-            }
-        }
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        accessor.reset(buffer);
-        int tupleCount = accessor.getTupleCount();
-        for (int i = 0; i < tupleCount; i++) {
-            tuple.reset(accessor, i);
-            invIndex.bulkLoadAddTuple(tuple, bulkLoadCtx);
-        }
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        try {
-            invIndex.endBulkLoad(bulkLoadCtx);
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        } finally {
-            try {
-                btreeDataflowHelper.deinit();
-            } finally {
-                invIndexDataflowHelper.deinit();
-            }
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        writer.fail();
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorDescriptor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorDescriptor.java
deleted file mode 100644
index 01d1a97..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorDescriptor.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public class InvertedIndexCreateOperatorDescriptor extends AbstractInvertedIndexOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    public InvertedIndexCreateOperatorDescriptor(IOperatorDescriptorRegistry spec,
-            IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
-            IFileSplitProvider invListsFileSplitProvider, IIndexRegistryProvider<IIndex> indexRegistryProvider,
-            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
-            ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
-            IBinaryTokenizerFactory tokenizerFactory, IIndexDataflowHelperFactory btreeDataflowHelperFactory,
-            IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 0, 0, null, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
-                indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
-                invListComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory, false, opCallbackProvider);
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new InvertedIndexCreateOperatorNodePushable(this, ctx, partition);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorNodePushable.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorNodePushable.java
deleted file mode 100644
index eb909d8..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorNodePushable.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-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.std.base.AbstractOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
-
-public class InvertedIndexCreateOperatorNodePushable extends AbstractOperatorNodePushable {
-    private final TreeIndexDataflowHelper btreeDataflowHelper;
-    private final InvertedIndexDataflowHelper invIndexDataflowHelper;
-
-    public InvertedIndexCreateOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
-            IHyracksTaskContext ctx, int partition) {
-        btreeDataflowHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory()
-                .createIndexDataflowHelper(opDesc, ctx, partition);
-        invIndexDataflowHelper = new InvertedIndexDataflowHelper(btreeDataflowHelper, opDesc, ctx, partition);
-    }
-
-    @Override
-    public void deinitialize() throws HyracksDataException {
-    }
-
-    @Override
-    public int getInputArity() {
-        return 0;
-    }
-
-    @Override
-    public IFrameWriter getInputFrameWriter(int index) {
-        return null;
-    }
-
-    @Override
-    public void initialize() throws HyracksDataException {
-    	// BTree.
-    	try {
-    		btreeDataflowHelper.init(true);
-    	} finally {
-    		btreeDataflowHelper.deinit();
-    	}
-    	// Inverted Index.
-    	try {
-    		invIndexDataflowHelper.init(true);
-    	} finally {
-    		invIndexDataflowHelper.deinit();
-    	}
-    }
-
-    @Override
-    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java
deleted file mode 100644
index 6d0ff38..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeElementInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
-
-public final class InvertedIndexDataflowHelper extends IndexDataflowHelper {
-    private final TreeIndexDataflowHelper btreeDataflowHelper;
-
-    public InvertedIndexDataflowHelper(TreeIndexDataflowHelper btreeDataflowHelper, IIndexOperatorDescriptor opDesc,
-            IHyracksTaskContext ctx, int partition) {
-        super(opDesc, ctx, partition);
-        this.btreeDataflowHelper = btreeDataflowHelper;
-    }
-
-    public FileReference getFilereference() {
-        AbstractInvertedIndexOperatorDescriptor invIndexOpDesc = (AbstractInvertedIndexOperatorDescriptor) opDesc;
-        IFileSplitProvider fileSplitProvider = invIndexOpDesc.getInvListsFileSplitProvider();
-        return fileSplitProvider.getFileSplits()[partition].getLocalFile();
-    }
-
-    @Override
-    public IIndex createIndexInstance() throws HyracksDataException {
-        IInvertedIndexOperatorDescriptor invIndexOpDesc = (IInvertedIndexOperatorDescriptor) opDesc;
-        // Assumes btreeDataflowHelper.init() has already been called.
-        BTree btree = (BTree) btreeDataflowHelper.getIndex();
-        IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(
-                invIndexOpDesc.getInvListsTypeTraits());
-        return new InvertedIndex(opDesc.getStorageManager().getBufferCache(ctx), btree,
-                invIndexOpDesc.getInvListsTypeTraits(), invIndexOpDesc.getInvListsComparatorFactories(),
-                invListBuilder);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
deleted file mode 100644
index 497dfb5..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-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.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public class InvertedIndexSearchOperatorDescriptor extends AbstractInvertedIndexOperatorDescriptor {
-    private static final long serialVersionUID = 1L;
-
-    private final int queryField;
-    private final IInvertedIndexSearchModifierFactory searchModifierFactory;
-
-    public InvertedIndexSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, int queryField,
-            IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
-            IFileSplitProvider invListsFileSplitProvider, IIndexRegistryProvider<IIndex> indexRegistryProvider,
-            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
-            ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
-            IIndexDataflowHelperFactory btreeDataflowHelperFactory, IBinaryTokenizerFactory queryTokenizerFactory,
-            IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc, boolean retainInput,
-            IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 1, 1, recDesc, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
-                indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
-                invListComparatorFactories, queryTokenizerFactory, btreeDataflowHelperFactory, retainInput,
-                opCallbackProvider);
-        this.queryField = queryField;
-        this.searchModifierFactory = searchModifierFactory;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
-        IInvertedIndexSearchModifier searchModifier = searchModifierFactory.createSearchModifier();
-        return new InvertedIndexSearchOperatorNodePushable(this, ctx, partition, queryField, searchModifier,
-                recordDescProvider);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java
deleted file mode 100644
index f3080f4..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.dataflow;
-
-import java.io.DataOutput;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-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;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-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.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndexSearchPredicate;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.OccurrenceThresholdPanicException;
-
-public class InvertedIndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-    private final TreeIndexDataflowHelper btreeDataflowHelper;
-    private final InvertedIndexDataflowHelper invIndexDataflowHelper;
-    private final int queryField;
-    private FrameTupleAccessor accessor;
-    private FrameTupleReference tuple;
-    private IRecordDescriptorProvider recordDescProvider;
-    private InvertedIndex invIndex;
-
-    private final InvertedIndexSearchPredicate searchPred;
-    private IIndexAccessor indexAccessor;
-    private IIndexCursor resultCursor;
-
-    private ByteBuffer writeBuffer;
-    private FrameTupleAppender appender;
-    private ArrayTupleBuilder tb;
-    private DataOutput dos;
-
-    private final AbstractInvertedIndexOperatorDescriptor opDesc;
-    private final boolean retainInput;
-
-    public InvertedIndexSearchOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
-            IHyracksTaskContext ctx, int partition, int queryField, IInvertedIndexSearchModifier searchModifier,
-            IRecordDescriptorProvider recordDescProvider) {
-        this.opDesc = opDesc;
-        btreeDataflowHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory()
-                .createIndexDataflowHelper(opDesc, ctx, partition);
-        invIndexDataflowHelper = new InvertedIndexDataflowHelper(btreeDataflowHelper, opDesc, ctx, partition);
-        this.queryField = queryField;
-        this.searchPred = new InvertedIndexSearchPredicate(opDesc.getTokenizerFactory().createTokenizer(), searchModifier);
-        this.recordDescProvider = recordDescProvider;
-        this.retainInput = invIndexDataflowHelper.getOperatorDescriptor().getRetainInput();
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(btreeDataflowHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
-        tuple = new FrameTupleReference();
-        // BTree.
-        try {
-            btreeDataflowHelper.init(false);
-        } catch (Exception e) {
-            // Cleanup in case of failure/
-            btreeDataflowHelper.deinit();
-            if (e instanceof HyracksDataException) {
-                throw (HyracksDataException) e;
-            } else {
-                throw new HyracksDataException(e);
-            }
-        }
-        // Inverted Index.
-        try {
-            invIndexDataflowHelper.init(false);
-            invIndex = (InvertedIndex) invIndexDataflowHelper.getIndex();
-        } catch (Exception e) {
-            // Cleanup in case of failure.
-            invIndexDataflowHelper.deinit();
-            if (e instanceof HyracksDataException) {
-                throw (HyracksDataException) e;
-            } else {
-                throw new HyracksDataException(e);
-            }
-        }
-
-        writeBuffer = btreeDataflowHelper.getHyracksTaskContext().allocateFrame();
-        tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
-        dos = tb.getDataOutput();
-        appender = new FrameTupleAppender(btreeDataflowHelper.getHyracksTaskContext().getFrameSize());
-        appender.reset(writeBuffer, true);
-
-        indexAccessor = invIndex.createAccessor();
-        //InvertedIndex.InvertedIndexAccessor accessor =  
-        resultCursor = indexAccessor.createSearchCursor();
-        writer.open();
-    }
-
-    private void writeSearchResults() throws Exception {
-        while (resultCursor.hasNext()) {
-            resultCursor.next();
-            tb.reset();
-            if (retainInput) {
-                for (int i = 0; i < tuple.getFieldCount(); i++) {
-                	dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
-                    tb.addFieldEndOffset();
-                }
-            }
-            ITupleReference invListElement = resultCursor.getTuple();
-            int invListFields = opDesc.getInvListsTypeTraits().length;
-            for (int i = 0; i < invListFields; i++) {
-                dos.write(invListElement.getFieldData(i), invListElement.getFieldStart(i),
-                        invListElement.getFieldLength(i));
-                tb.addFieldEndOffset();
-            }
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                FrameUtils.flushFrame(writeBuffer, writer);
-                appender.reset(writeBuffer, true);
-                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    throw new IllegalStateException();
-                }
-            }
-        }
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        accessor.reset(buffer);
-        int tupleCount = accessor.getTupleCount();
-        try {
-            for (int i = 0; i < tupleCount; i++) {
-                tuple.reset(accessor, i);
-                searchPred.setQueryTuple(tuple);
-                searchPred.setQueryFieldIndex(queryField);
-                try {
-                    resultCursor.reset();
-                    indexAccessor.search(resultCursor, searchPred);
-                    writeSearchResults();
-                } catch (OccurrenceThresholdPanicException e) {
-                    // Ignore panic cases for now.
-                }
-            }
-        } catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        writer.fail();
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        try {
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(writeBuffer, writer);
-            }
-            writer.close();
-        } finally {
-            try {
-                btreeDataflowHelper.deinit();
-            } finally {
-                invIndexDataflowHelper.deinit();
-            }
-        }
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
deleted file mode 100644
index c4ab601..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
-
-public class FixedSizeElementInvertedListBuilder implements IInvertedListBuilder {
-    private final int listElementSize;
-    private int listSize = 0;
-
-    private byte[] targetBuf;
-    private int pos;
-
-    public FixedSizeElementInvertedListBuilder(ITypeTraits[] invListFields) {
-        int tmp = 0;
-        for (int i = 0; i < invListFields.length; i++) {
-            tmp += invListFields[i].getFixedLength();
-        }
-        listElementSize = tmp;
-    }
-
-    @Override
-    public boolean startNewList(ITupleReference tuple, int tokenField) {
-        if (pos + listElementSize >= targetBuf.length)
-            return false;
-        else {
-            listSize = 0;
-            return true;
-        }
-    }
-
-    @Override
-    public boolean appendElement(ITupleReference tuple, int numTokenFields, int numElementFields) {
-        if (pos + listElementSize >= targetBuf.length)
-            return false;
-
-        for (int i = 0; i < numElementFields; i++) {
-            int field = numTokenFields + i;
-            System.arraycopy(tuple.getFieldData(field), tuple.getFieldStart(field), targetBuf, pos,
-                    tuple.getFieldLength(field));
-        }
-
-        listSize++;
-        pos += listElementSize;
-
-        return true;
-    }
-
-    @Override
-    public void setTargetBuffer(byte[] targetBuf, int startPos) {
-        this.targetBuf = targetBuf;
-        this.pos = startPos;
-    }
-
-    @Override
-    public int getListSize() {
-        return listSize;
-    }
-
-    @Override
-    public int getPos() {
-        return pos;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
deleted file mode 100644
index 446f171..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
+++ /dev/null
@@ -1,280 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.invertedindex.impls;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-
-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.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
-
-public class FixedSizeElementInvertedListCursor implements IInvertedListCursor {
-
-    private final IBufferCache bufferCache;
-    private final int fileId;
-    private final int elementSize;
-    private int currentElementIx;
-    private int currentOff;
-    private int currentPageIx;
-
-    private int startPageId;
-    private int endPageId;
-    private int startOff;
-    private int numElements;
-
-    private final FixedSizeTupleReference tuple;
-    private ICachedPage[] pages = new ICachedPage[10];
-    private int[] elementIndexes = new int[10];
-
-    public FixedSizeElementInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields) {
-        this.bufferCache = bufferCache;
-        this.fileId = fileId;
-        this.currentElementIx = 0;
-        this.currentPageIx = 0;
-
-        int tmp = 0;
-        for (int i = 0; i < invListFields.length; i++) {
-            tmp += invListFields[i].getFixedLength();
-        }
-        elementSize = tmp;
-        this.currentOff = -elementSize;
-        this.tuple = new FixedSizeTupleReference(invListFields);
-    }
-
-    @Override
-    public boolean hasNext() {
-        if (currentElementIx < numElements)
-            return true;
-        else
-            return false;
-    }
-
-    @Override
-    public void next() {
-        if (currentOff + 2 * elementSize >= bufferCache.getPageSize()) {
-            currentPageIx++;
-            currentOff = 0;
-        } else {
-            currentOff += elementSize;
-        }
-
-        currentElementIx++;
-        tuple.reset(pages[currentPageIx].getBuffer().array(), currentOff);
-    }
-
-    @Override
-    public void pinPagesAsync() {
-        // TODO: implement
-    }
-
-    @Override
-    public void pinPagesSync() throws HyracksDataException {
-        int pix = 0;
-        for (int i = startPageId; i <= endPageId; i++) {
-            pages[pix] = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i), false);
-            pages[pix].acquireReadLatch();
-            pix++;
-        }
-    }
-
-    @Override
-    public void unpinPages() throws HyracksDataException {
-        int numPages = endPageId - startPageId + 1;
-        for (int i = 0; i < numPages; i++) {
-            pages[i].releaseReadLatch();
-            bufferCache.unpin(pages[i]);
-        }
-    }
-
-    @Override
-    public void positionCursor(int elementIx) {
-        int numPages = endPageId - startPageId + 1;
-
-        currentPageIx = binarySearch(elementIndexes, 0, numPages, elementIx);
-        if (currentPageIx < 0) {
-            throw new IndexOutOfBoundsException("Requested index: " + elementIx + " from array with numElements: "
-                    + numElements);
-        }
-
-        if (currentPageIx == 0) {
-            currentOff = startOff + elementIx * elementSize;
-        } else {
-            int relativeElementIx = elementIx - elementIndexes[currentPageIx - 1] - 1;
-            currentOff = relativeElementIx * elementSize;
-        }
-
-        currentElementIx = elementIx;
-        tuple.reset(pages[currentPageIx].getBuffer().array(), currentOff);
-    }
-
-    @Override
-    public boolean containsKey(ITupleReference searchTuple, MultiComparator invListCmp) {
-        int mid;
-        int begin = 0;
-        int end = numElements - 1;
-
-        while (begin <= end) {
-            mid = (begin + end) / 2;
-            positionCursor(mid);
-            int cmp = invListCmp.compare(searchTuple, tuple);
-            if (cmp < 0) {
-                end = mid - 1;
-            } else if (cmp > 0) {
-                begin = mid + 1;
-            } else {
-                return true;
-            }
-        }
-
-        return false;
-    }
-
-    @Override
-    public void reset(int startPageId, int endPageId, int startOff, int numElements) {
-        this.startPageId = startPageId;
-        this.endPageId = endPageId;
-        this.startOff = startOff;
-        this.numElements = numElements;
-        this.currentElementIx = 0;
-        this.currentPageIx = 0;
-        this.currentOff = startOff - elementSize;
-
-        int numPages = endPageId - startPageId + 1;
-        if (numPages > pages.length) {
-            pages = new ICachedPage[endPageId - startPageId + 1];
-            elementIndexes = new int[endPageId - startPageId + 1];
-        }
-
-        // fill elementIndexes
-        // first page
-        int cumulElements = (bufferCache.getPageSize() - startOff) / elementSize;
-        elementIndexes[0] = cumulElements - 1;
-
-        // middle, full pages
-        for (int i = 1; i < numPages - 1; i++) {
-            elementIndexes[i] = elementIndexes[i - 1] + (bufferCache.getPageSize() / elementSize);
-        }
-
-        // last page
-        elementIndexes[numPages - 1] = numElements - 1;
-    }
-
-    @Override
-    public String printInvList(ISerializerDeserializer[] serdes) throws HyracksDataException {
-        int oldCurrentOff = currentOff;
-        int oldCurrentPageId = currentPageIx;
-        int oldCurrentElementIx = currentElementIx;
-
-        currentOff = startOff - elementSize;
-        currentPageIx = 0;
-        currentElementIx = 0;
-
-        StringBuilder strBuilder = new StringBuilder();
-
-        int count = 0;
-        while (hasNext()) {
-            next();
-            count++;
-            for (int i = 0; i < tuple.getFieldCount(); i++) {
-                ByteArrayInputStream inStream = new ByteArrayInputStream(tuple.getFieldData(i), tuple.getFieldStart(i),
-                        tuple.getFieldLength(i));
-                DataInput dataIn = new DataInputStream(inStream);
-                Object o = serdes[i].deserialize(dataIn);
-                strBuilder.append(o.toString());
-                if (i + 1 < tuple.getFieldCount())
-                    strBuilder.append(",");
-            }
-            strBuilder.append(" ");
-        }
-
-        // reset previous state
-        currentOff = oldCurrentOff;
-        currentPageIx = oldCurrentPageId;
-        currentElementIx = oldCurrentElementIx;
-
-        return strBuilder.toString();
-    }
-
-    public String printCurrentElement(ISerializerDeserializer[] serdes) throws HyracksDataException {
-        StringBuilder strBuilder = new StringBuilder();
-        for (int i = 0; i < tuple.getFieldCount(); i++) {
-            ByteArrayInputStream inStream = new ByteArrayInputStream(tuple.getFieldData(i), tuple.getFieldStart(i),
-                    tuple.getFieldLength(i));
-            DataInput dataIn = new DataInputStream(inStream);
-            Object o = serdes[i].deserialize(dataIn);
-            strBuilder.append(o.toString());
-            if (i + 1 < tuple.getFieldCount())
-                strBuilder.append(",");
-        }
-        return strBuilder.toString();
-    }
-
-    private int binarySearch(int[] arr, int arrStart, int arrLength, int key) {
-        int mid;
-        int begin = arrStart;
-        int end = arrStart + arrLength - 1;
-
-        while (begin <= end) {
-            mid = (begin + end) / 2;
-            int cmp = (key - arr[mid]);
-            if (cmp < 0) {
-                end = mid - 1;
-            } else if (cmp > 0) {
-                begin = mid + 1;
-            } else {
-                return mid;
-            }
-        }
-
-        if (begin > arr.length - 1)
-            return -1;
-        if (key < arr[begin])
-            return begin;
-        else
-            return -1;
-    }
-
-    @Override
-    public int compareTo(IInvertedListCursor invListCursor) {
-        return numElements - invListCursor.getNumElements();
-    }
-
-    @Override
-    public int getEndPageId() {
-        return endPageId;
-    }
-
-    @Override
-    public int getNumElements() {
-        return numElements;
-    }
-
-    @Override
-    public int getStartOff() {
-        return startOff;
-    }
-
-    @Override
-    public int getStartPageId() {
-        return startPageId;
-    }
-
-    public int getOffset() {
-        return currentOff;
-    }
-
-    public ICachedPage getPage() {
-        return pages[currentPageIx];
-    }
-
-    @Override
-    public ITupleReference getTuple() {
-        return tuple;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java
deleted file mode 100644
index cbedc45..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAccessor.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-
-public class FixedSizeFrameTupleAccessor implements IFrameTupleAccessor {
-
-    private final int frameSize;
-    private ByteBuffer buffer;
-
-    private final ITypeTraits[] fields;
-    private final int[] fieldStartOffsets;
-    private final int tupleSize;
-
-    public FixedSizeFrameTupleAccessor(int frameSize, ITypeTraits[] fields) {
-        this.frameSize = frameSize;
-        this.fields = fields;
-        this.fieldStartOffsets = new int[fields.length];
-        this.fieldStartOffsets[0] = 0;
-        for (int i = 1; i < fields.length; i++) {
-            fieldStartOffsets[i] = fieldStartOffsets[i - 1] + fields[i - 1].getFixedLength();
-        }
-
-        int tmp = 0;
-        for (int i = 0; i < fields.length; i++) {
-            tmp += fields[i].getFixedLength();
-        }
-        tupleSize = tmp;
-    }
-
-    @Override
-    public ByteBuffer getBuffer() {
-        return buffer;
-    }
-
-    @Override
-    public int getFieldCount() {
-        return fields.length;
-    }
-
-    @Override
-    public int getFieldEndOffset(int tupleIndex, int fIdx) {
-        return getTupleStartOffset(tupleIndex) + fieldStartOffsets[fIdx] + fields[fIdx].getFixedLength();
-    }
-
-    @Override
-    public int getFieldLength(int tupleIndex, int fIdx) {
-        return fields[fIdx].getFixedLength();
-    }
-
-    @Override
-    public int getFieldSlotsLength() {
-        return 0;
-    }
-
-    @Override
-    public int getFieldStartOffset(int tupleIndex, int fIdx) {
-        return tupleIndex * tupleSize + fieldStartOffsets[fIdx];
-    }
-
-    @Override
-    public int getTupleCount() {
-        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
-    }
-
-    @Override
-    public int getTupleEndOffset(int tupleIndex) {
-        return getFieldEndOffset(tupleIndex, fields.length - 1);
-    }
-
-    @Override
-    public int getTupleStartOffset(int tupleIndex) {
-        return tupleIndex * tupleSize;
-    }
-
-    @Override
-    public void reset(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java
deleted file mode 100644
index 489ec2e..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeFrameTupleAppender.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-
-public class FixedSizeFrameTupleAppender {
-
-    private static final int TUPLE_COUNT_SIZE = 4;
-    private final int frameSize;
-    private final int tupleSize;
-    private ByteBuffer buffer;
-    private int tupleCount;
-    private int tupleDataEndOffset;
-
-    public FixedSizeFrameTupleAppender(int frameSize, ITypeTraits[] fields) {
-        this.frameSize = frameSize;
-        int tmp = 0;
-        for (int i = 0; i < fields.length; i++) {
-            tmp += fields[i].getFixedLength();
-        }
-        tupleSize = tmp;
-    }
-
-    public void reset(ByteBuffer buffer, boolean clear) {
-        this.buffer = buffer;
-        if (clear) {
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), 0);
-            tupleCount = 0;
-            tupleDataEndOffset = 0;
-        }
-    }
-
-    public boolean append(byte[] bytes, int offset) {
-        if (tupleDataEndOffset + tupleSize + TUPLE_COUNT_SIZE <= frameSize) {
-            System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset, tupleSize);
-            tupleDataEndOffset += tupleSize;
-            tupleCount++;
-            return true;
-        }
-        return false;
-    }
-
-    public boolean append(byte[] bytes, int offset, int length) {
-        if (tupleDataEndOffset + length + TUPLE_COUNT_SIZE <= frameSize) {
-            System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset, length);
-            tupleDataEndOffset += length;
-            return true;
-        }
-        return false;
-    }
-
-    public boolean append(int fieldValue) {
-        if (tupleDataEndOffset + 4 + TUPLE_COUNT_SIZE <= frameSize) {
-            buffer.putInt(tupleDataEndOffset, fieldValue);
-            tupleDataEndOffset += 4;
-            tupleCount++;
-            return true;
-        }
-        return false;
-    }
-
-    public boolean append(long fieldValue) {
-        if (tupleDataEndOffset + 8 + TUPLE_COUNT_SIZE <= frameSize) {
-            buffer.putLong(tupleDataEndOffset, fieldValue);
-            tupleDataEndOffset += 8;
-            tupleCount++;
-            return true;
-        }
-        return false;
-    }
-
-    public boolean append(char fieldValue) {
-        if (tupleDataEndOffset + 2 + TUPLE_COUNT_SIZE <= frameSize) {
-            buffer.putLong(tupleDataEndOffset, fieldValue);
-            tupleDataEndOffset += 2;
-            tupleCount++;
-            return true;
-        }
-        return false;
-    }
-
-    public boolean append(byte fieldValue) {
-        if (tupleDataEndOffset + 1 + TUPLE_COUNT_SIZE <= frameSize) {
-            buffer.put(tupleDataEndOffset, fieldValue);
-            tupleDataEndOffset += 1;
-            tupleCount++;
-            return true;
-        }
-        return false;
-    }
-
-    // returns true if an entire tuple fits
-    // returns false otherwise
-    public boolean hasSpace() {
-        return tupleDataEndOffset + tupleSize + TUPLE_COUNT_SIZE <= frameSize;
-    }
-
-    public void incrementTupleCount(int count) {
-        buffer.putInt(FrameHelper.getTupleCountOffset(frameSize),
-                buffer.getInt(FrameHelper.getTupleCountOffset(frameSize)) + count);
-    }
-
-    public int getTupleCount() {
-        return tupleCount;
-    }
-
-    public ByteBuffer getBuffer() {
-        return buffer;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java
deleted file mode 100644
index 0656d69..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeTupleReference.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-
-public class FixedSizeTupleReference implements ITupleReference {
-
-    private final ITypeTraits[] typeTraits;
-    private final int[] fieldStartOffsets;
-    private byte[] data;
-    private int startOff;
-
-    public FixedSizeTupleReference(ITypeTraits[] typeTraits) {
-        this.typeTraits = typeTraits;
-        this.fieldStartOffsets = new int[typeTraits.length];
-        this.fieldStartOffsets[0] = 0;
-        for (int i = 1; i < typeTraits.length; i++) {
-            fieldStartOffsets[i] = fieldStartOffsets[i - 1] + typeTraits[i - 1].getFixedLength();
-        }
-    }
-
-    public void reset(byte[] data, int startOff) {
-        this.data = data;
-        this.startOff = startOff;
-    }
-
-    @Override
-    public int getFieldCount() {
-        return typeTraits.length;
-    }
-
-    @Override
-    public byte[] getFieldData(int fIdx) {
-        return data;
-    }
-
-    @Override
-    public int getFieldLength(int fIdx) {
-        return typeTraits[fIdx].getFixedLength();
-    }
-
-    @Override
-    public int getFieldStart(int fIdx) {
-        return startOff + fieldStartOffsets[fIdx];
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
deleted file mode 100644
index 3525fc3..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ /dev/null
@@ -1,358 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-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.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-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.IIndexBulkLoadContext;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.common.api.IndexType;
-import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearcher;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
-
-/**
- * An inverted index consists of two files: 1. a file storing (paginated)
- * inverted lists 2. a BTree-file mapping from tokens to inverted lists.
- * Implemented features: bulk loading and searching (based on T-Occurrence) Not
- * implemented features: updates (insert/update/delete) Limitations: a query
- * cannot exceed the size of a Hyracks frame.
- */
-public class InvertedIndex implements IIndex {
-    private final IHyracksCommonContext ctx = new DefaultHyracksCommonContext();
-
-    private BTree btree;
-    private int rootPageId = 0;
-    private IBufferCache bufferCache;
-    private int fileId;
-    private final ITypeTraits[] invListTypeTraits;
-    private final IBinaryComparatorFactory[] invListCmpFactories;
-    private final IInvertedListBuilder invListBuilder;
-    private final int numTokenFields;
-    private final int numInvListKeys;
-
-    public InvertedIndex(IBufferCache bufferCache, BTree btree, ITypeTraits[] invListTypeTraits,
-            IBinaryComparatorFactory[] invListCmpFactories, IInvertedListBuilder invListBuilder) {
-        this.bufferCache = bufferCache;
-        this.btree = btree;
-        this.invListTypeTraits = invListTypeTraits;
-        this.invListCmpFactories = invListCmpFactories;
-        this.invListBuilder = invListBuilder;
-        this.numTokenFields = btree.getComparatorFactories().length;
-        this.numInvListKeys = invListCmpFactories.length;
-    }
-
-    @Override
-    public void open(int fileId) {
-        this.fileId = fileId;
-    }
-
-    @Override
-    public void create(int indexFileId) throws HyracksDataException {
-    }
-
-    @Override
-    public void close() {
-        this.fileId = -1;
-    }
-
-    public boolean openCursor(ITreeIndexCursor btreeCursor, RangePredicate btreePred, ITreeIndexAccessor btreeAccessor,
-            IInvertedListCursor invListCursor) throws HyracksDataException, IndexException {
-        btreeAccessor.search(btreeCursor, btreePred);
-        boolean ret = false;
-        try {
-            if (btreeCursor.hasNext()) {
-                btreeCursor.next();
-                ITupleReference frameTuple = btreeCursor.getTuple();
-                // Hardcoded mapping of btree fields
-                int startPageId = IntegerSerializerDeserializer.getInt(frameTuple.getFieldData(1),
-                        frameTuple.getFieldStart(1));
-                int endPageId = IntegerSerializerDeserializer.getInt(frameTuple.getFieldData(2),
-                        frameTuple.getFieldStart(2));
-                int startOff = IntegerSerializerDeserializer.getInt(frameTuple.getFieldData(3),
-                        frameTuple.getFieldStart(3));
-                int numElements = IntegerSerializerDeserializer.getInt(frameTuple.getFieldData(4),
-                        frameTuple.getFieldStart(4));
-                invListCursor.reset(startPageId, endPageId, startOff, numElements);
-                ret = true;
-            } else {
-                invListCursor.reset(0, 0, 0, 0);
-            }
-        } finally {
-            btreeCursor.close();
-            btreeCursor.reset();
-        }
-        return ret;
-    }
-
-    @Override
-    public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws TreeIndexException, HyracksDataException {
-        InvertedIndexBulkLoadContext ctx = new InvertedIndexBulkLoadContext(fillFactor);
-        ctx.init(rootPageId, fileId);
-        return ctx;
-    }
-
-    /**
-     * Assumptions:
-     * The first btree.getMultiComparator().getKeyFieldCount() fields in tuple
-     * are btree keys (e.g., a string token).
-     * The next invListCmp.getKeyFieldCount() fields in tuple are keys of the
-     * inverted list (e.g., primary key).
-     * Key fields of inverted list are fixed size.
-     */
-    @Override
-    public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException {
-        InvertedIndexBulkLoadContext ctx = (InvertedIndexBulkLoadContext) ictx;
-        boolean firstElement = ctx.lastTupleBuilder.getSize() == 0;
-        boolean startNewList = firstElement;
-        if (!firstElement) {
-            // If the current and the last token don't match, we start a new list.
-            ctx.lastTuple.reset(ctx.lastTupleBuilder.getFieldEndOffsets(), ctx.lastTupleBuilder.getByteArray());
-            startNewList = ctx.tokenCmp.compare(tuple, ctx.lastTuple) != 0;
-        }
-        if (startNewList) {
-            if (!firstElement) {
-                // Create entry in btree for last inverted list.
-                createAndInsertBTreeTuple(ctx);
-            }
-            if (!invListBuilder.startNewList(tuple, numTokenFields)) {
-                ctx.pinNextPage();
-                invListBuilder.setTargetBuffer(ctx.currentPage.getBuffer().array(), 0);
-                if (!invListBuilder.startNewList(tuple, numTokenFields)) {
-                    throw new IllegalStateException("Failed to create first inverted list.");
-                }
-            }
-            ctx.currentInvListStartPageId = ctx.currentPageId;
-            ctx.currentInvListStartOffset = invListBuilder.getPos();
-        } else {
-            if (ctx.invListCmp.compare(tuple, ctx.lastTuple, numTokenFields) == 0) {
-                // Duplicate inverted-list element.
-                return;
-            }
-        }
-
-        // Append to current inverted list.
-        if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
-            ctx.pinNextPage();
-            invListBuilder.setTargetBuffer(ctx.currentPage.getBuffer().array(), 0);
-            if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
-                throw new IllegalStateException(
-                        "Failed to append element to inverted list after switching to a new page.");
-            }
-        }
-
-        // Remember last tuple by creating a copy.
-        // TODO: This portion can be optimized by only copying the token when it changes, and using the last appended inverted-list element as a reference.
-        ctx.lastTupleBuilder.reset();
-        for (int i = 0; i < tuple.getFieldCount(); i++) {
-            ctx.lastTupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
-        }
-    }
-
-    private void createAndInsertBTreeTuple(InvertedIndexBulkLoadContext ctx) throws HyracksDataException {
-        // Build tuple.        
-        ctx.btreeTupleBuilder.reset();
-        ctx.btreeTupleBuilder.addField(ctx.lastTuple.getFieldData(0), ctx.lastTuple.getFieldStart(0),
-                ctx.lastTuple.getFieldLength(0));
-        ctx.btreeTupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, ctx.currentInvListStartPageId);
-        ctx.btreeTupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, ctx.currentPageId);
-        ctx.btreeTupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, ctx.currentInvListStartOffset);
-        ctx.btreeTupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, invListBuilder.getListSize());
-        // Reset tuple reference and add it.
-        ctx.btreeTupleReference.reset(ctx.btreeTupleBuilder.getFieldEndOffsets(), ctx.btreeTupleBuilder.getByteArray());
-        btree.bulkLoadAddTuple(ctx.btreeTupleReference, ctx.btreeBulkLoadCtx);
-    }
-
-    @Override
-    public void endBulkLoad(IIndexBulkLoadContext ictx) throws HyracksDataException {
-        // Create entry in btree for last inverted list.
-        InvertedIndexBulkLoadContext ctx = (InvertedIndexBulkLoadContext) ictx;
-        if (ctx.lastTuple.getFieldData(0) != null) {
-            createAndInsertBTreeTuple(ctx);
-        }
-        btree.endBulkLoad(ctx.btreeBulkLoadCtx);
-        ctx.deinit();
-    }
-
-    public final class InvertedIndexBulkLoadContext implements IIndexBulkLoadContext {
-        private final ArrayTupleBuilder btreeTupleBuilder;
-        private final ArrayTupleReference btreeTupleReference;
-        private final float btreeFillFactor;
-        private IIndexBulkLoadContext btreeBulkLoadCtx;
-
-        private int currentInvListStartPageId;
-        private int currentInvListStartOffset;
-        private final ArrayTupleBuilder lastTupleBuilder;
-        private final ArrayTupleReference lastTuple;
-
-        private int currentPageId;
-        private ICachedPage currentPage;
-        private final MultiComparator tokenCmp;
-        private final MultiComparator invListCmp;
-
-        public InvertedIndexBulkLoadContext(float btreeFillFactor) {
-            this.tokenCmp = MultiComparator.create(btree.getComparatorFactories());
-            this.invListCmp = MultiComparator.create(invListCmpFactories);
-            this.btreeTupleBuilder = new ArrayTupleBuilder(btree.getFieldCount());
-            this.btreeTupleReference = new ArrayTupleReference();
-            this.btreeFillFactor = btreeFillFactor;
-            this.lastTupleBuilder = new ArrayTupleBuilder(numTokenFields + numInvListKeys);
-            this.lastTuple = new ArrayTupleReference();
-        }
-
-        public void init(int startPageId, int fileId) throws HyracksDataException, TreeIndexException {
-            btreeBulkLoadCtx = btree.beginBulkLoad(btreeFillFactor);
-            currentPageId = startPageId;
-            currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
-            currentPage.acquireWriteLatch();
-            invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
-        }
-
-        public void deinit() throws HyracksDataException {
-            if (currentPage != null) {
-                currentPage.releaseWriteLatch();
-                bufferCache.unpin(currentPage);
-            }
-        }
-
-        public void pinNextPage() throws HyracksDataException {
-            currentPage.releaseWriteLatch();
-            bufferCache.unpin(currentPage);
-            currentPageId++;
-            currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
-            currentPage.acquireWriteLatch();
-        }
-    }
-
-    @Override
-    public IBufferCache getBufferCache() {
-        return bufferCache;
-    }
-
-    public int getInvListsFileId() {
-        return fileId;
-    }
-
-    public IBinaryComparatorFactory[] getInvListElementCmpFactories() {
-        return invListCmpFactories;
-    }
-
-    public ITypeTraits[] getTypeTraits() {
-        return invListTypeTraits;
-    }
-
-    public BTree getBTree() {
-        return btree;
-    }
-
-    public class InvertedIndexAccessor implements IIndexAccessor {
-        private final IInvertedIndexSearcher searcher;
-
-        public InvertedIndexAccessor(InvertedIndex index) {
-            this.searcher = new TOccurrenceSearcher(ctx, index);
-        }
-
-        @Override
-        public void insert(ITupleReference tuple) throws HyracksDataException, IndexException {
-            // TODO Auto-generated method stub
-        }
-
-        @Override
-        public void update(ITupleReference tuple) throws HyracksDataException, IndexException {
-            // TODO Auto-generated method stub
-        }
-
-        @Override
-        public void delete(ITupleReference tuple) throws HyracksDataException, IndexException {
-            // TODO Auto-generated method stub
-        }
-
-        @Override
-        public void upsert(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
-            // TODO Auto-generated method stub
-        }
-
-        @Override
-        public IIndexCursor createSearchCursor() {
-            return new InvertedIndexSearchCursor(searcher);
-        }
-
-        @Override
-        public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
-                IndexException {
-            searcher.search((InvertedIndexSearchCursor) cursor, (InvertedIndexSearchPredicate) searchPred);
-        }
-
-        public IInvertedIndexSearcher getSearcher() {
-            return searcher;
-        }
-    }
-
-    @Override
-    public IIndexAccessor createAccessor() {
-        return new InvertedIndexAccessor(this);
-    }
-
-    @Override
-    public IndexType getIndexType() {
-        return IndexType.INVERTED;
-    }
-
-    // This is just a dummy hyracks context for allocating frames for temporary
-    // results during inverted index searches.
-    // TODO: In the future we should use the real HyracksTaskContext to track
-    // frame usage.
-    private class DefaultHyracksCommonContext implements IHyracksCommonContext {
-        private final int FRAME_SIZE = 32768;
-
-        @Override
-        public int getFrameSize() {
-            return FRAME_SIZE;
-        }
-
-        @Override
-        public IIOManager getIOManager() {
-            return null;
-        }
-
-        @Override
-        public ByteBuffer allocateFrame() {
-            return ByteBuffer.allocate(FRAME_SIZE);
-        }
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexException.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexException.java
deleted file mode 100644
index 9762a74..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexException.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-
-public class InvertedIndexException extends IndexException {
-    private static final long serialVersionUID = 1L;
-
-    public InvertedIndexException(String msg) {
-        super(msg);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchCursor.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchCursor.java
deleted file mode 100644
index 1eeb576..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchCursor.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearcher;
-
-public class InvertedIndexSearchCursor implements IIndexCursor {
-
-    private List<ByteBuffer> resultBuffers;
-    private int numResultBuffers;
-    private int currentBufferIndex = 0;
-    private int tupleIndex = 0;
-    private final IInvertedIndexSearcher invIndexSearcher;
-    private final IFrameTupleAccessor fta;
-    private final FixedSizeTupleReference resultTuple;
-    
-    public InvertedIndexSearchCursor(IInvertedIndexSearcher invIndexSearcher) {
-        this.invIndexSearcher = invIndexSearcher;
-        this.fta = invIndexSearcher.createResultFrameTupleAccessor();
-        this.resultTuple = (FixedSizeTupleReference) invIndexSearcher.createResultTupleReference();
-    }
-
-    @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        currentBufferIndex = 0;
-        tupleIndex = 0;
-        resultBuffers = invIndexSearcher.getResultBuffers();
-        numResultBuffers = invIndexSearcher.getNumValidResultBuffers();
-        if (numResultBuffers > 0) {
-            fta.reset(resultBuffers.get(0));
-        }
-    }
-    
-    @Override
-    public boolean hasNext() {
-        if (currentBufferIndex < numResultBuffers && tupleIndex < fta.getTupleCount()) {
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public void next() {
-        resultTuple.reset(fta.getBuffer().array(), fta.getTupleStartOffset(tupleIndex));
-        tupleIndex++;
-        if (tupleIndex >= fta.getTupleCount()) {
-            if (currentBufferIndex + 1 < numResultBuffers) {
-                currentBufferIndex++;
-                fta.reset(resultBuffers.get(currentBufferIndex));
-                tupleIndex = 0;
-            }
-        }
-    }
-
-    @Override
-    public ITupleReference getTuple() {
-        return resultTuple;
-    }
-
-    @Override
-    public void reset() {
-        currentBufferIndex = 0;
-        tupleIndex = 0;
-        invIndexSearcher.reset();
-        resultBuffers = invIndexSearcher.getResultBuffers();
-        numResultBuffers = invIndexSearcher.getNumValidResultBuffers();
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        currentBufferIndex = 0;
-        tupleIndex = 0;
-        resultBuffers = null;
-        numResultBuffers = 0;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchPredicate.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchPredicate.java
deleted file mode 100644
index 9e9a2c3..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchPredicate.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-
-public class InvertedIndexSearchPredicate implements ISearchPredicate {
-    private static final long serialVersionUID = 1L;
-
-    private ITupleReference queryTuple;
-    private int queryFieldIndex;
-    private final IBinaryTokenizer queryTokenizer;
-    private final IInvertedIndexSearchModifier searchModifier;    
-    
-    public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer, IInvertedIndexSearchModifier searchModifier) {
-        this.queryTokenizer = queryTokenizer;
-        this.searchModifier = searchModifier;
-    }
-    
-    public void setQueryTuple(ITupleReference queryTuple) {
-        this.queryTuple = queryTuple;
-    }
-    
-    public ITupleReference getQueryTuple() {
-        return queryTuple;
-    }
-    
-    public void setQueryFieldIndex(int queryFieldIndex) {
-        this.queryFieldIndex = queryFieldIndex;
-    }
-    
-    public int getQueryFieldIndex() {
-        return queryFieldIndex;
-    }
-    
-    public IInvertedIndexSearchModifier getSearchModifier() {
-        return searchModifier;
-    }
-    
-    public IBinaryTokenizer getQueryTokenizer() {
-        return queryTokenizer;
-    }
-    
-    @Override
-    public MultiComparator getLowKeyComparator() {
-        // TODO: This doesn't make sense for an inverted index. Change ISearchPredicate interface.
-        return null;
-    }
-
-    @Override
-    public MultiComparator getHighKeyComparator() {
-        // TODO: This doesn't make sense for an inverted index. Change ISearchPredicate interface.
-        return null;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/OccurrenceThresholdPanicException.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/OccurrenceThresholdPanicException.java
deleted file mode 100644
index b0e737c..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/OccurrenceThresholdPanicException.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-public class OccurrenceThresholdPanicException extends InvertedIndexException {
-    private static final long serialVersionUID = 1L;
-
-    public OccurrenceThresholdPanicException(String msg) {
-        super(msg);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
deleted file mode 100644
index af5dad3..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ /dev/null
@@ -1,547 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-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.data.std.primitive.IntegerPointable;
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-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.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-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.IndexException;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearcher;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-
-// TODO: The search procedure is rather confusing regarding cursor positions, hasNext() calls etc.
-// Needs an overhaul some time.
-public class TOccurrenceSearcher implements IInvertedIndexSearcher {
-
-    protected final IHyracksCommonContext ctx;
-    protected final FixedSizeFrameTupleAppender resultFrameTupleApp;
-    protected final FixedSizeFrameTupleAccessor resultFrameTupleAcc;
-    protected final FixedSizeTupleReference resultTuple;
-    protected final int invListKeyLength;
-    protected int currentNumResults;
-
-    protected List<ByteBuffer> newResultBuffers = new ArrayList<ByteBuffer>();
-    protected List<ByteBuffer> prevResultBuffers = new ArrayList<ByteBuffer>();
-    protected List<ByteBuffer> swap = null;
-    protected int maxResultBufIdx = 0;
-
-    protected final ITreeIndexFrame leafFrame;
-    protected final ITreeIndexFrame interiorFrame;
-    protected final ITreeIndexCursor btreeCursor;
-    protected final FrameTupleReference searchKey = new FrameTupleReference();
-    protected final RangePredicate btreePred = new RangePredicate(null, null, true, true, null, null);
-    protected final ITreeIndexAccessor btreeAccessor;
-
-    protected RecordDescriptor queryTokenRecDesc = new RecordDescriptor(
-            new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
-    protected ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(queryTokenRecDesc.getFieldCount());
-    protected GrowableArray queryTokenFieldData = queryTokenBuilder.getFieldData();
-    protected FrameTupleAppender queryTokenAppender;
-    protected ByteBuffer queryTokenFrame;
-
-    protected final InvertedIndex invIndex;
-    protected final MultiComparator invListCmp;
-    protected final ITypeTraits[] invListFieldsWithCount;
-    protected int occurrenceThreshold;
-
-    protected final int cursorCacheSize = 10;
-    protected List<IInvertedListCursor> invListCursorCache = new ArrayList<IInvertedListCursor>(cursorCacheSize);
-    protected List<IInvertedListCursor> invListCursors = new ArrayList<IInvertedListCursor>(cursorCacheSize);
-
-    public TOccurrenceSearcher(IHyracksCommonContext ctx, InvertedIndex invIndex) {
-        this.ctx = ctx;
-        this.invIndex = invIndex;
-        this.invListCmp = MultiComparator.create(invIndex.getInvListElementCmpFactories());
-
-        leafFrame = invIndex.getBTree().getLeafFrameFactory().createFrame();
-        interiorFrame = invIndex.getBTree().getInteriorFrameFactory().createFrame();
-
-        btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
-        ITypeTraits[] invListFields = invIndex.getTypeTraits();
-        invListFieldsWithCount = new ITypeTraits[invListFields.length + 1];
-        int tmp = 0;
-        for (int i = 0; i < invListFields.length; i++) {
-            invListFieldsWithCount[i] = invListFields[i];
-            tmp += invListFields[i].getFixedLength();
-        }
-        // using an integer for counting occurrences
-        invListFieldsWithCount[invListFields.length] = IntegerPointable.TYPE_TRAITS;
-        invListKeyLength = tmp;
-
-        resultFrameTupleApp = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), invListFieldsWithCount);
-        resultFrameTupleAcc = new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), invListFieldsWithCount);
-        resultTuple = new FixedSizeTupleReference(invListFieldsWithCount);
-        newResultBuffers.add(ctx.allocateFrame());
-        prevResultBuffers.add(ctx.allocateFrame());
-
-        MultiComparator searchCmp = MultiComparator.create(invIndex.getBTree().getComparatorFactories());
-        btreePred.setLowKeyComparator(searchCmp);
-        btreePred.setHighKeyComparator(searchCmp);
-        btreePred.setLowKey(searchKey, true);
-        btreePred.setHighKey(searchKey, true);
-
-        // pre-create cursor objects
-        for (int i = 0; i < cursorCacheSize; i++) {
-            invListCursorCache.add(new FixedSizeElementInvertedListCursor(invIndex.getBufferCache(), invIndex
-                    .getInvListsFileId(), invIndex.getTypeTraits()));
-        }
-
-        queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
-        queryTokenFrame = ctx.allocateFrame();
-
-        btreeAccessor = invIndex.getBTree().createAccessor();
-        currentNumResults = 0;
-    }
-
-    public void reset() {
-        for (ByteBuffer b : newResultBuffers) {
-            resultFrameTupleApp.reset(b, true);
-        }
-        for (ByteBuffer b : prevResultBuffers) {
-            resultFrameTupleApp.reset(b, true);
-        }
-        currentNumResults = 0;
-    }
-
-    public void search(InvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred) throws HyracksDataException, IndexException {
-        ITupleReference queryTuple = searchPred.getQueryTuple();
-        int queryFieldIndex = searchPred.getQueryFieldIndex();
-        IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier();
-        IBinaryTokenizer queryTokenizer = searchPred.getQueryTokenizer();
-        
-        queryTokenAppender.reset(queryTokenFrame, true);                
-        queryTokenizer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
-                queryTuple.getFieldLength(queryFieldIndex));
-
-        while (queryTokenizer.hasNext()) {
-            queryTokenizer.next();
-            queryTokenBuilder.reset();
-            try {
-                IToken token = queryTokenizer.getToken();
-                token.serializeToken(queryTokenFieldData);
-                queryTokenBuilder.addFieldEndOffset();
-                // WARNING: assuming one frame is big enough to hold all tokens
-                queryTokenAppender.append(queryTokenBuilder.getFieldEndOffsets(), queryTokenBuilder.getByteArray(), 0,
-                        queryTokenBuilder.getSize());
-            } catch (IOException e) {
-                throw new HyracksDataException(e);
-            }
-        }
-
-        FrameTupleAccessor queryTokenAccessor = new FrameTupleAccessor(ctx.getFrameSize(), queryTokenRecDesc);
-        queryTokenAccessor.reset(queryTokenFrame);
-        int numQueryTokens = queryTokenAccessor.getTupleCount();
-
-        // expand cursor cache if necessary
-        if (numQueryTokens > invListCursorCache.size()) {
-            int diff = numQueryTokens - invListCursorCache.size();
-            for (int i = 0; i < diff; i++) {
-                invListCursorCache.add(new FixedSizeElementInvertedListCursor(invIndex.getBufferCache(), invIndex
-                        .getInvListsFileId(), invIndex.getTypeTraits()));
-            }
-        }
-
-        invListCursors.clear();
-        for (int i = 0; i < numQueryTokens; i++) {
-            searchKey.reset(queryTokenAccessor, i);
-            invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursorCache.get(i));
-            invListCursors.add(invListCursorCache.get(i));
-        }
-
-        occurrenceThreshold = searchModifier.getOccurrenceThreshold(invListCursors);
-
-        // TODO: deal with panic cases properly
-        if (occurrenceThreshold <= 0) {
-            throw new OccurrenceThresholdPanicException("Merge Threshold is <= 0. Failing Search.");
-        }
-
-        int numPrefixLists = searchModifier.getPrefixLists(invListCursors);
-        maxResultBufIdx = mergePrefixLists(numPrefixLists, numQueryTokens);
-        maxResultBufIdx = mergeSuffixLists(numPrefixLists, numQueryTokens, maxResultBufIdx);
-
-        resultCursor.open(null, searchPred);
-    }
-
-    protected int mergePrefixLists(int numPrefixTokens, int numQueryTokens) throws HyracksDataException {
-        int maxPrevBufIdx = 0;
-        for (int i = 0; i < numPrefixTokens; i++) {
-            swap = prevResultBuffers;
-            prevResultBuffers = newResultBuffers;
-            newResultBuffers = swap;
-            currentNumResults = 0;
-
-            invListCursors.get(i).pinPagesSync();
-            maxPrevBufIdx = mergePrefixList(invListCursors.get(i), prevResultBuffers, maxPrevBufIdx, newResultBuffers);
-            invListCursors.get(i).unpinPages();
-        }
-        return maxPrevBufIdx;
-    }
-
-    protected int mergeSuffixLists(int numPrefixTokens, int numQueryTokens, int maxPrevBufIdx) throws HyracksDataException {
-        for (int i = numPrefixTokens; i < numQueryTokens; i++) {
-            swap = prevResultBuffers;
-            prevResultBuffers = newResultBuffers;
-            newResultBuffers = swap;
-
-            invListCursors.get(i).pinPagesSync();
-            int numInvListElements = invListCursors.get(i).getNumElements();
-            // should we binary search the next list or should we sort-merge it?
-            if (currentNumResults * Math.log(numInvListElements) < currentNumResults + numInvListElements) {
-                maxPrevBufIdx = mergeSuffixListProbe(invListCursors.get(i), prevResultBuffers, maxPrevBufIdx,
-                        newResultBuffers, i, numQueryTokens);
-            } else {
-                maxPrevBufIdx = mergeSuffixListScan(invListCursors.get(i), prevResultBuffers, maxPrevBufIdx,
-                        newResultBuffers, i, numQueryTokens);
-            }
-            invListCursors.get(i).unpinPages();
-        }
-        return maxPrevBufIdx;
-    }
-
-    protected int mergeSuffixListProbe(IInvertedListCursor invListCursor, List<ByteBuffer> prevResultBuffers,
-            int maxPrevBufIdx, List<ByteBuffer> newResultBuffers, int invListIx, int numQueryTokens) {
-
-        int newBufIdx = 0;
-        ByteBuffer newCurrentBuffer = newResultBuffers.get(0);
-
-        int prevBufIdx = 0;
-        ByteBuffer prevCurrentBuffer = prevResultBuffers.get(0);
-
-        int resultTidx = 0;
-
-        currentNumResults = 0;
-
-        resultFrameTupleAcc.reset(prevCurrentBuffer);
-        resultFrameTupleApp.reset(newCurrentBuffer, true);
-
-        while (resultTidx < resultFrameTupleAcc.getTupleCount()) {
-
-            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
-            int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                    resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
-
-            if (invListCursor.containsKey(resultTuple, invListCmp)) {
-                count++;
-                newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-            } else {
-                if (count + numQueryTokens - invListIx > occurrenceThreshold) {
-                    newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-                }
-            }
-
-            resultTidx++;
-            if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
-                prevBufIdx++;
-                if (prevBufIdx <= maxPrevBufIdx) {
-                    prevCurrentBuffer = prevResultBuffers.get(prevBufIdx);
-                    resultFrameTupleAcc.reset(prevCurrentBuffer);
-                    resultTidx = 0;
-                }
-            }
-        }
-
-        return newBufIdx;
-    }
-
-    protected int mergeSuffixListScan(IInvertedListCursor invListCursor, List<ByteBuffer> prevResultBuffers,
-            int maxPrevBufIdx, List<ByteBuffer> newResultBuffers, int invListIx, int numQueryTokens) {
-        int newBufIdx = 0;
-        ByteBuffer newCurrentBuffer = newResultBuffers.get(0);
-
-        int prevBufIdx = 0;
-        ByteBuffer prevCurrentBuffer = prevResultBuffers.get(0);
-
-        boolean advanceCursor = true;
-        boolean advancePrevResult = false;
-        int resultTidx = 0;
-        currentNumResults = 0;
-        
-        resultFrameTupleAcc.reset(prevCurrentBuffer);
-        resultFrameTupleApp.reset(newCurrentBuffer, true);
-
-        int invListTidx = 0;
-        int invListNumTuples = invListCursor.getNumElements();
-
-        if (invListCursor.hasNext())
-            invListCursor.next();
-
-        while (invListTidx < invListNumTuples && resultTidx < resultFrameTupleAcc.getTupleCount()) {
-
-            ITupleReference invListTuple = invListCursor.getTuple();
-
-            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
-
-            int cmp = invListCmp.compare(invListTuple, resultTuple);
-            if (cmp == 0) {
-                int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                        resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)) + 1;
-                newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-                advanceCursor = true;
-                advancePrevResult = true;
-            } else {
-                if (cmp < 0) {
-                    advanceCursor = true;
-                    advancePrevResult = false;
-                } else {
-                    int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                            resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
-                    if (count + numQueryTokens - invListIx > occurrenceThreshold) {
-                        newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-                    }
-                    advanceCursor = false;
-                    advancePrevResult = true;
-                }
-            }
-
-            if (advancePrevResult) {
-                resultTidx++;
-                if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
-                    prevBufIdx++;
-                    if (prevBufIdx <= maxPrevBufIdx) {
-                        prevCurrentBuffer = prevResultBuffers.get(prevBufIdx);
-                        resultFrameTupleAcc.reset(prevCurrentBuffer);
-                        resultTidx = 0;
-                    }
-                }
-            }
-
-            if (advanceCursor) {
-                invListTidx++;
-                if (invListCursor.hasNext()) {
-                	invListCursor.next();
-                }
-            }
-        }
-
-        // append remaining elements from previous result set
-        while (resultTidx < resultFrameTupleAcc.getTupleCount()) {
-
-            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
-
-            int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                    resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
-            if (count + numQueryTokens - invListIx > occurrenceThreshold) {
-                newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-            }
-
-            resultTidx++;
-            if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
-                prevBufIdx++;
-                if (prevBufIdx <= maxPrevBufIdx) {
-                    prevCurrentBuffer = prevResultBuffers.get(prevBufIdx);
-                    resultFrameTupleAcc.reset(prevCurrentBuffer);
-                    resultTidx = 0;
-                }
-            }
-        }
-
-        return newBufIdx;
-    }
-
-    protected int mergePrefixList(IInvertedListCursor invListCursor, List<ByteBuffer> prevResultBuffers,
-            int maxPrevBufIdx, List<ByteBuffer> newResultBuffers) {
-        int newBufIdx = 0;
-        ByteBuffer newCurrentBuffer = newResultBuffers.get(0);
-
-        int prevBufIdx = 0;
-        ByteBuffer prevCurrentBuffer = prevResultBuffers.get(0);
-
-        boolean advanceCursor = true;
-        boolean advancePrevResult = false;
-        int resultTidx = 0;
-
-        resultFrameTupleAcc.reset(prevCurrentBuffer);
-        resultFrameTupleApp.reset(newCurrentBuffer, true);
-
-        int invListTidx = 0;
-        int invListNumTuples = invListCursor.getNumElements();
-
-        if (invListCursor.hasNext())
-            invListCursor.next();
-
-        while (invListTidx < invListNumTuples && resultTidx < resultFrameTupleAcc.getTupleCount()) {
-
-            ITupleReference invListTuple = invListCursor.getTuple();
-            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
-
-            int cmp = invListCmp.compare(invListTuple, resultTuple);
-            if (cmp == 0) {
-                int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                        resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)) + 1;
-                newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-                advanceCursor = true;
-                advancePrevResult = true;
-            } else {
-                if (cmp < 0) {
-                    int count = 1;
-                    newBufIdx = appendTupleToNewResults(invListTuple, count, newBufIdx);
-                    advanceCursor = true;
-                    advancePrevResult = false;
-                } else {
-                    int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                            resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
-                    newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-                    advanceCursor = false;
-                    advancePrevResult = true;
-                }
-            }
-
-            if (advancePrevResult) {
-                resultTidx++;
-                if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
-                    prevBufIdx++;
-                    if (prevBufIdx <= maxPrevBufIdx) {
-                        prevCurrentBuffer = prevResultBuffers.get(prevBufIdx);
-                        resultFrameTupleAcc.reset(prevCurrentBuffer);
-                        resultTidx = 0;
-                    }
-                }
-            }
-
-            if (advanceCursor) {
-                invListTidx++;
-                if (invListCursor.hasNext()) {
-                	invListCursor.next();
-                }
-            }
-        }
-
-        // append remaining new elements from inverted list
-        while (invListTidx < invListNumTuples) {
-            ITupleReference invListTuple = invListCursor.getTuple();
-            newBufIdx = appendTupleToNewResults(invListTuple, 1, newBufIdx);
-            invListTidx++;
-            if (invListCursor.hasNext()) {
-                invListCursor.next();
-            }
-        }
-
-        // append remaining elements from previous result set
-        while (resultTidx < resultFrameTupleAcc.getTupleCount()) {
-
-            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
-
-            int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                    resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
-            newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-
-            resultTidx++;
-            if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
-                prevBufIdx++;
-                if (prevBufIdx <= maxPrevBufIdx) {
-                    prevCurrentBuffer = prevResultBuffers.get(prevBufIdx);
-                    resultFrameTupleAcc.reset(prevCurrentBuffer);
-                    resultTidx = 0;
-                }
-            }
-        }
-
-        return newBufIdx;
-    }
-
-    protected int appendTupleToNewResults(ITupleReference tuple, int newCount, int newBufIdx) {
-        ByteBuffer newCurrentBuffer = newResultBuffers.get(newBufIdx);
-
-        if (!resultFrameTupleApp.hasSpace()) {
-            newBufIdx++;
-            if (newBufIdx >= newResultBuffers.size()) {
-                newResultBuffers.add(ctx.allocateFrame());
-            }
-            newCurrentBuffer = newResultBuffers.get(newBufIdx);
-            resultFrameTupleApp.reset(newCurrentBuffer, true);
-        }
-
-        // append key
-        if (!resultFrameTupleApp.append(tuple.getFieldData(0), tuple.getFieldStart(0), invListKeyLength)) {
-            throw new IllegalStateException();
-        }
-
-        // append new count
-        if (!resultFrameTupleApp.append(newCount)) {
-            throw new IllegalStateException();
-        }
-
-        resultFrameTupleApp.incrementTupleCount(1);
-
-        currentNumResults++;
-
-        return newBufIdx;
-    }
-
-    public IFrameTupleAccessor createResultFrameTupleAccessor() {
-        return new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), invListFieldsWithCount);
-    }
-
-    public ITupleReference createResultTupleReference() {
-        return new FixedSizeTupleReference(invListFieldsWithCount);
-    }
-
-    @Override
-    public List<ByteBuffer> getResultBuffers() {
-        return newResultBuffers;
-    }
-
-    @Override
-    public int getNumValidResultBuffers() {
-        return maxResultBufIdx + 1;
-    }
-
-    public int getOccurrenceThreshold() {
-        return occurrenceThreshold;
-    }
-    
-    public void printNewResults(int maxResultBufIdx) {
-        StringBuffer strBuffer = new StringBuffer();
-        for (int i = 0; i <= maxResultBufIdx; i++) {
-            ByteBuffer testBuf = newResultBuffers.get(i);
-            resultFrameTupleAcc.reset(testBuf);
-            for (int j = 0; j < resultFrameTupleAcc.getTupleCount(); j++) {
-                strBuffer.append(IntegerSerializerDeserializer.getInt(resultFrameTupleAcc.getBuffer().array(),
-                        resultFrameTupleAcc.getFieldStartOffset(j, 0)) + ",");
-                strBuffer.append(IntegerSerializerDeserializer.getInt(resultFrameTupleAcc.getBuffer().array(),
-                        resultFrameTupleAcc.getFieldStartOffset(j, 1)) + " ");
-            }
-        }
-        System.out.println(strBuffer.toString());
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
deleted file mode 100644
index 957ecc0..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-
-public class TOccurrenceSearcherSuffixProbeOnly extends TOccurrenceSearcher {
-
-	protected final MultiComparator invListCmp;
-	
-    public TOccurrenceSearcherSuffixProbeOnly(IHyracksTaskContext ctx, InvertedIndex invIndex) {
-        super(ctx, invIndex);
-        this.invListCmp = MultiComparator.create(invIndex.getInvListElementCmpFactories());
-    }
-
-    protected int mergeSuffixLists(int numPrefixTokens, int numQueryTokens, int maxPrevBufIdx) throws HyracksDataException {
-        for (int i = numPrefixTokens; i < numQueryTokens; i++) {
-            swap = prevResultBuffers;
-            prevResultBuffers = newResultBuffers;
-            newResultBuffers = swap;
-            currentNumResults = 0;
-
-            invListCursors.get(i).pinPagesSync();
-            maxPrevBufIdx = mergeSuffixListProbe(invListCursors.get(i), prevResultBuffers, maxPrevBufIdx,
-                    newResultBuffers, i, numQueryTokens);
-            invListCursors.get(i).unpinPages();
-        }
-        return maxPrevBufIdx;
-    }
-
-    protected int mergeSuffixListProbe(IInvertedListCursor invListCursor, List<ByteBuffer> prevResultBuffers,
-            int maxPrevBufIdx, List<ByteBuffer> newResultBuffers, int invListIx, int numQueryTokens) {
-
-        int newBufIdx = 0;
-        ByteBuffer newCurrentBuffer = newResultBuffers.get(0);
-
-        int prevBufIdx = 0;
-        ByteBuffer prevCurrentBuffer = prevResultBuffers.get(0);
-
-        int resultTidx = 0;
-
-        resultFrameTupleAcc.reset(prevCurrentBuffer);
-        resultFrameTupleApp.reset(newCurrentBuffer, true);
-
-        while (resultTidx < resultFrameTupleAcc.getTupleCount()) {
-
-            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
-            int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                    resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
-
-            if (invListCursor.containsKey(resultTuple, invListCmp)) {
-                count++;
-                newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-            } else {
-                if (count + numQueryTokens - invListIx > occurrenceThreshold) {
-                    newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-                }
-            }
-
-            resultTidx++;
-            if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
-                prevBufIdx++;
-                if (prevBufIdx <= maxPrevBufIdx) {
-                    prevCurrentBuffer = prevResultBuffers.get(prevBufIdx);
-                    resultFrameTupleAcc.reset(prevCurrentBuffer);
-                    resultTidx = 0;
-                }
-            }
-        }
-
-        return newBufIdx;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
deleted file mode 100644
index bd9bd60..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.impls;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-
-public class TOccurrenceSearcherSuffixScanOnly extends TOccurrenceSearcher {
-
-	protected final MultiComparator invListCmp;
-	
-    public TOccurrenceSearcherSuffixScanOnly(IHyracksTaskContext ctx, InvertedIndex invIndex) {
-        super(ctx, invIndex);
-        this.invListCmp = MultiComparator.create(invIndex.getInvListElementCmpFactories());
-    }
-
-    protected int mergeSuffixLists(int numPrefixTokens, int numQueryTokens, int maxPrevBufIdx) throws HyracksDataException {
-        for (int i = numPrefixTokens; i < numQueryTokens; i++) {
-            swap = prevResultBuffers;
-            prevResultBuffers = newResultBuffers;
-            newResultBuffers = swap;
-            currentNumResults = 0;
-
-            invListCursors.get(i).pinPagesSync();
-            maxPrevBufIdx = mergeSuffixListScan(invListCursors.get(i), prevResultBuffers, maxPrevBufIdx,
-                    newResultBuffers, i, numQueryTokens);
-            invListCursors.get(i).unpinPages();
-        }
-        return maxPrevBufIdx;
-    }
-
-    protected int mergeSuffixListScan(IInvertedListCursor invListCursor, List<ByteBuffer> prevResultBuffers,
-            int maxPrevBufIdx, List<ByteBuffer> newResultBuffers, int invListIx, int numQueryTokens) {
-
-        int newBufIdx = 0;
-        ByteBuffer newCurrentBuffer = newResultBuffers.get(0);
-
-        int prevBufIdx = 0;
-        ByteBuffer prevCurrentBuffer = prevResultBuffers.get(0);
-
-        boolean advanceCursor = true;
-        boolean advancePrevResult = false;
-        int resultTidx = 0;
-
-        resultFrameTupleAcc.reset(prevCurrentBuffer);
-        resultFrameTupleApp.reset(newCurrentBuffer, true);
-
-        while (invListCursor.hasNext() && resultTidx < resultFrameTupleAcc.getTupleCount()) {
-
-            if (advanceCursor)
-                invListCursor.next();
-
-            ITupleReference invListTuple = invListCursor.getTuple();
-
-            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
-
-            int cmp = invListCmp.compare(invListTuple, resultTuple);
-            if (cmp == 0) {
-                int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                        resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)) + 1;
-                newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-                advanceCursor = true;
-                advancePrevResult = true;
-            } else {
-                if (cmp < 0) {
-                    advanceCursor = true;
-                    advancePrevResult = false;
-                } else {
-                    int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                            resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
-                    if (count + numQueryTokens - invListIx > occurrenceThreshold) {
-                        newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-                    }
-                    advanceCursor = false;
-                    advancePrevResult = true;
-                }
-            }
-
-            if (advancePrevResult) {
-                resultTidx++;
-                if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
-                    prevBufIdx++;
-                    if (prevBufIdx <= maxPrevBufIdx) {
-                        prevCurrentBuffer = prevResultBuffers.get(prevBufIdx);
-                        resultFrameTupleAcc.reset(prevCurrentBuffer);
-                        resultTidx = 0;
-                    }
-                }
-            }
-        }
-
-        // append remaining elements from previous result set
-        while (resultTidx < resultFrameTupleAcc.getTupleCount()) {
-
-            int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
-                    resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
-            newBufIdx = appendTupleToNewResults(resultTuple, count, newBufIdx);
-
-            resultTidx++;
-            if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
-                prevBufIdx++;
-                if (prevBufIdx <= maxPrevBufIdx) {
-                    prevCurrentBuffer = prevResultBuffers.get(prevBufIdx);
-                    resultFrameTupleAcc.reset(prevCurrentBuffer);
-                    resultTidx = 0;
-                }
-            }
-        }
-
-        return newBufIdx;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifier.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifier.java
deleted file mode 100644
index 55945be..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifier.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.searchmodifiers;
-
-import java.util.Collections;
-import java.util.List;
-
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-
-public class ConjunctiveSearchModifier implements IInvertedIndexSearchModifier {
-
-    @Override
-    public int getOccurrenceThreshold(List<IInvertedListCursor> invListCursors) {
-        return invListCursors.size();
-    }
-
-    @Override
-    public int getPrefixLists(List<IInvertedListCursor> invListCursors) {
-        Collections.sort(invListCursors);
-        return 1;
-    }
-
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifierFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifierFactory.java
deleted file mode 100644
index 0db6008..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifierFactory.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.searchmodifiers;
-
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
-
-public class ConjunctiveSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public IInvertedIndexSearchModifier createSearchModifier() {
-        return new ConjunctiveSearchModifier();
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifier.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifier.java
deleted file mode 100644
index ac109b6..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifier.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.searchmodifiers;
-
-import java.util.Collections;
-import java.util.List;
-
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-
-public class EditDistanceSearchModifier implements IInvertedIndexSearchModifier {
-
-    private int gramLength;
-    private int edThresh;
-
-    public EditDistanceSearchModifier(int gramLength, int edThresh) {
-        this.gramLength = gramLength;
-        this.edThresh = edThresh;
-    }
-
-    @Override
-    public int getOccurrenceThreshold(List<IInvertedListCursor> invListCursors) {
-        return invListCursors.size() - edThresh * gramLength;
-    }
-
-    @Override
-    public int getPrefixLists(List<IInvertedListCursor> invListCursors) {
-        Collections.sort(invListCursors);
-        return invListCursors.size() - getOccurrenceThreshold(invListCursors) + 1;
-    }
-
-    public int getGramLength() {
-        return gramLength;
-    }
-
-    public void setGramLength(int gramLength) {
-        this.gramLength = gramLength;
-    }
-
-    public int getEdThresh() {
-        return edThresh;
-    }
-
-    public void setEdThresh(int edThresh) {
-        this.edThresh = edThresh;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifierFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifierFactory.java
deleted file mode 100644
index 128d9db..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifierFactory.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.searchmodifiers;
-
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
-
-public class EditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private final int gramLength;
-    private final int edThresh;
-    
-    public EditDistanceSearchModifierFactory(int gramLength, int edThresh) {
-        this.gramLength = gramLength;
-        this.edThresh = edThresh;
-    }
-    
-    @Override
-    public IInvertedIndexSearchModifier createSearchModifier() {
-        return new EditDistanceSearchModifier(gramLength, edThresh);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifier.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifier.java
deleted file mode 100644
index 0a4961d..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifier.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.searchmodifiers;
-
-import java.util.Collections;
-import java.util.List;
-
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-
-public class JaccardSearchModifier implements IInvertedIndexSearchModifier {
-
-    private float jaccThresh;
-
-    public JaccardSearchModifier(float jaccThresh) {
-        this.jaccThresh = jaccThresh;
-    }
-
-    @Override
-    public int getOccurrenceThreshold(List<IInvertedListCursor> invListCursors) {
-        return Math.max((int) Math.floor((float) invListCursors.size() * jaccThresh), 1);
-    }
-
-    @Override
-    public int getPrefixLists(List<IInvertedListCursor> invListCursors) {
-        Collections.sort(invListCursors);
-        if (invListCursors.size() == 0) {
-            return 0;
-        }
-        return invListCursors.size() - getOccurrenceThreshold(invListCursors) + 1;
-    }
-
-    public float getJaccThresh() {
-        return jaccThresh;
-    }
-
-    public void setJaccThresh(float jaccThresh) {
-        this.jaccThresh = jaccThresh;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifierFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifierFactory.java
deleted file mode 100644
index bd27c03..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifierFactory.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.searchmodifiers;
-
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
-
-public class JaccardSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    private final float jaccThresh;
-
-    public JaccardSearchModifierFactory(float jaccThresh) {
-        this.jaccThresh = jaccThresh;
-    }
-
-    @Override
-    public IInvertedIndexSearchModifier createSearchModifier() {
-        return new JaccardSearchModifier(jaccThresh);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
deleted file mode 100644
index c2992f5..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-
-public abstract class AbstractUTF8StringBinaryTokenizer implements IBinaryTokenizer {
-
-    protected byte[] data;
-    protected int start;
-    protected int length;
-    protected int tokenLength;
-    protected int index;
-    protected int utf8Length;
-
-    protected final IntArray tokensStart;
-    protected final IntArray tokensLength;
-    protected final IToken token;
-
-    protected final boolean ignoreTokenCount;
-    protected final boolean sourceHasTypeTag;
-
-    public AbstractUTF8StringBinaryTokenizer(boolean ignoreTokenCount, boolean sourceHasTypeTag,
-            ITokenFactory tokenFactory) {
-        this.ignoreTokenCount = ignoreTokenCount;
-        this.sourceHasTypeTag = sourceHasTypeTag;
-        if (!ignoreTokenCount) {
-            tokensStart = new IntArray();
-            tokensLength = new IntArray();
-        } else {
-            tokensStart = null;
-            tokensLength = null;
-        }
-        token = tokenFactory.createToken();
-    }
-
-    @Override
-    public IToken getToken() {
-        return token;
-    }
-
-    @Override
-    public void reset(byte[] data, int start, int length) {
-        this.start = start;
-        index = this.start;
-        if (sourceHasTypeTag) {
-            index++; // skip type tag
-        }
-        utf8Length = UTF8StringPointable.getUTFLength(data, index);
-        index += 2; // skip utf8 length indicator
-        this.data = data;
-        this.length = length + start;
-
-        tokenLength = 0;
-        if (!ignoreTokenCount) {
-            tokensStart.reset();
-            tokensLength.reset();
-        }
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
deleted file mode 100644
index 2f60952..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-
-public abstract class AbstractUTF8Token implements IToken {
-    public static final int GOLDEN_RATIO_32 = 0x09e3779b9;
-
-    protected int length;
-    protected int tokenLength;
-    protected int start;
-    protected int tokenCount;
-    protected byte[] data;
-    protected final byte tokenTypeTag;
-    protected final byte countTypeTag;
-
-    public AbstractUTF8Token() {
-        tokenTypeTag = -1;
-        countTypeTag = -1;
-    }
-
-    public AbstractUTF8Token(byte tokenTypeTag, byte countTypeTag) {
-        this.tokenTypeTag = tokenTypeTag;
-        this.countTypeTag = countTypeTag;
-    }
-
-    @Override
-    public byte[] getData() {
-        return data;
-    }
-
-    @Override
-    public int getLength() {
-        return length;
-    }
-
-    public int getLowerCaseUTF8Len(int size) {
-        int lowerCaseUTF8Len = 0;
-        int pos = start;
-        for (int i = 0; i < size; i++) {
-            char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
-            lowerCaseUTF8Len += UTF8StringPointable.getModifiedUTF8Len(c);
-            pos += UTF8StringPointable.charSize(data, pos);
-        }
-        return lowerCaseUTF8Len;
-    }
-
-    @Override
-    public int getStart() {
-        return start;
-    }
-
-    @Override
-    public int getTokenLength() {
-        return tokenLength;
-    }
-
-    public void handleCountTypeTag(DataOutput dos) throws IOException {
-        if (countTypeTag > 0) {
-            dos.write(countTypeTag);
-        }
-    }
-
-    public void handleTokenTypeTag(DataOutput dos) throws IOException {
-        if (tokenTypeTag > 0) {
-            dos.write(tokenTypeTag);
-        }
-    }
-
-    @Override
-    public void reset(byte[] data, int start, int length, int tokenLength, int tokenCount) {
-        this.data = data;
-        this.start = start;
-        this.length = length;
-        this.tokenLength = tokenLength;
-        this.tokenCount = tokenCount;
-    }
-
-    @Override
-    public void serializeTokenCount(GrowableArray out) throws IOException {
-        handleCountTypeTag(out.getDataOutput());
-        out.getDataOutput().writeInt(tokenCount);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8TokenFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8TokenFactory.java
deleted file mode 100644
index 3b0b82d..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8TokenFactory.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-public abstract class AbstractUTF8TokenFactory implements ITokenFactory {
-	private static final long serialVersionUID = 1L;
-	protected final byte tokenTypeTag;
-	protected final byte countTypeTag;
-
-	public AbstractUTF8TokenFactory() {
-		tokenTypeTag = -1;
-		countTypeTag = -1;
-	}
-
-	public AbstractUTF8TokenFactory(byte tokenTypeTag, byte countTypeTag) {
-		this.tokenTypeTag = tokenTypeTag;
-		this.countTypeTag = countTypeTag;
-	}
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
deleted file mode 100644
index 9dacde6..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-
-public class DelimitedUTF8StringBinaryTokenizer extends AbstractUTF8StringBinaryTokenizer {
-
-    public DelimitedUTF8StringBinaryTokenizer(boolean ignoreTokenCount, boolean sourceHasTypeTag,
-            ITokenFactory tokenFactory) {
-        super(ignoreTokenCount, sourceHasTypeTag, tokenFactory);
-    }
-
-    @Override
-    public boolean hasNext() {
-        // skip delimiters
-        while (index < length && isSeparator(UTF8StringPointable.charAt(data, index))) {
-            index += UTF8StringPointable.charSize(data, index);
-        }
-        return index < length;
-    }
-
-    private boolean isSeparator(char c) {
-        return !(Character.isLetterOrDigit(c) || Character.getType(c) == Character.OTHER_LETTER || Character.getType(c) == Character.OTHER_NUMBER);
-    }
-
-    @Override
-    public void next() {
-        tokenLength = 0;
-        int currentTokenStart = index;
-        while (index < length && !isSeparator(UTF8StringPointable.charAt(data, index))) {
-            index += UTF8StringPointable.charSize(data, index);
-            tokenLength++;
-        }
-        int tokenCount = 1;
-        if (tokenLength > 0 && !ignoreTokenCount) {
-            // search if we got the same token before
-            for (int i = 0; i < tokensStart.length(); ++i) {
-                if (tokenLength == tokensLength.get(i)) {
-                    int tokenStart = tokensStart.get(i);
-                    tokenCount++; // assume we found it
-                    int offset = 0;
-                    int currLength = 0;
-                    while (currLength < tokenLength) {
-                        // case insensitive comparison
-                        if (Character.toLowerCase(UTF8StringPointable.charAt(data, currentTokenStart + offset)) != Character
-                                .toLowerCase(UTF8StringPointable.charAt(data, tokenStart + offset))) {
-                            tokenCount--;
-                            break;
-                        }
-                        offset += UTF8StringPointable.charSize(data, currentTokenStart + offset);
-                        currLength++;
-                    }
-                }
-            }
-            // add the new token to the list of seen tokens
-            tokensStart.add(currentTokenStart);
-            tokensLength.add(tokenLength);
-        }
-
-        // set token
-        token.reset(data, currentTokenStart, index, tokenLength, tokenCount);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java
deleted file mode 100644
index 4a350b3..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-public class DelimitedUTF8StringBinaryTokenizerFactory implements
-		IBinaryTokenizerFactory {
-
-	private static final long serialVersionUID = 1L;
-	private final boolean ignoreTokenCount;
-	private final boolean sourceHasTypeTag;
-	private final ITokenFactory tokenFactory;
-
-	public DelimitedUTF8StringBinaryTokenizerFactory(boolean ignoreTokenCount,
-			boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
-		this.ignoreTokenCount = ignoreTokenCount;
-		this.sourceHasTypeTag = sourceHasTypeTag;
-		this.tokenFactory = tokenFactory;
-	}
-
-	@Override
-	public IBinaryTokenizer createTokenizer() {
-		return new DelimitedUTF8StringBinaryTokenizer(ignoreTokenCount,
-				sourceHasTypeTag, tokenFactory);
-	}
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
deleted file mode 100644
index a1a4354..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-
-public class HashedUTF8NGramToken extends UTF8NGramToken {
-    public HashedUTF8NGramToken(byte tokenTypeTag, byte countTypeTag) {
-        super(tokenTypeTag, countTypeTag);
-    }
-
-    @Override
-    public void serializeToken(GrowableArray out) throws IOException {
-        handleTokenTypeTag(out.getDataOutput());
-
-        int hash = GOLDEN_RATIO_32;
-
-        // pre chars
-        for (int i = 0; i < numPreChars; i++) {
-            hash ^= PRECHAR;
-            hash *= GOLDEN_RATIO_32;
-        }
-
-        // regular chars
-        int numRegGrams = tokenLength - numPreChars - numPostChars;
-        int pos = start;
-        for (int i = 0; i < numRegGrams; i++) {
-            hash ^= Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
-            hash *= GOLDEN_RATIO_32;
-            pos += UTF8StringPointable.charSize(data, pos);
-        }
-
-        // post chars
-        for (int i = 0; i < numPostChars; i++) {
-            hash ^= POSTCHAR;
-            hash *= GOLDEN_RATIO_32;
-        }
-
-        // token count
-        hash += tokenCount;
-
-        out.getDataOutput().writeInt(hash);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java
deleted file mode 100644
index 4a87793..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-public class HashedUTF8NGramTokenFactory extends AbstractUTF8TokenFactory {
-
-	private static final long serialVersionUID = 1L;
-
-	public HashedUTF8NGramTokenFactory() {
-		super();
-	}
-
-	public HashedUTF8NGramTokenFactory(byte tokenTypeTag, byte countTypeTag) {
-		super(tokenTypeTag, countTypeTag);
-	}
-
-	@Override
-	public IToken createToken() {
-		return new HashedUTF8NGramToken(tokenTypeTag, countTypeTag);
-	}
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
deleted file mode 100644
index 20405c6..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-
-public class HashedUTF8WordToken extends UTF8WordToken {
-
-    private int hash = 0;
-
-    public HashedUTF8WordToken(byte tokenTypeTag, byte countTypeTag) {
-        super(tokenTypeTag, countTypeTag);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (o == null) {
-            return false;
-        }
-        if (!(o instanceof IToken)) {
-            return false;
-        }
-        IToken t = (IToken) o;
-        if (t.getTokenLength() != tokenLength) {
-            return false;
-        }
-        int offset = 0;
-        for (int i = 0; i < tokenLength; i++) {
-            if (UTF8StringPointable.charAt(t.getData(), t.getStart() + offset) != UTF8StringPointable.charAt(data,
-                    start + offset)) {
-                return false;
-            }
-            offset += UTF8StringPointable.charSize(data, start + offset);
-        }
-        return true;
-    }
-
-    @Override
-    public int hashCode() {
-        return hash;
-    }
-
-    @Override
-    public void reset(byte[] data, int start, int length, int tokenLength, int tokenCount) {
-        super.reset(data, start, length, tokenLength, tokenCount);
-
-        // pre-compute hash value using JAQL-like string hashing
-        int pos = start;
-        hash = GOLDEN_RATIO_32;
-        for (int i = 0; i < tokenLength; i++) {
-            hash ^= Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
-            hash *= GOLDEN_RATIO_32;
-            pos += UTF8StringPointable.charSize(data, pos);
-        }
-        hash += tokenCount;
-    }
-
-    @Override
-    public void serializeToken(GrowableArray out) throws IOException {
-        if (tokenTypeTag > 0) {
-            out.getDataOutput().write(tokenTypeTag);
-        }
-
-        // serialize hash value
-        out.getDataOutput().writeInt(hash);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java
deleted file mode 100644
index 318f041..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-public class HashedUTF8WordTokenFactory extends AbstractUTF8TokenFactory {
-
-	private static final long serialVersionUID = 1L;
-
-	public HashedUTF8WordTokenFactory() {
-		super();
-	}
-
-	public HashedUTF8WordTokenFactory(byte tokenTypeTag, byte countTypeTag) {
-		super(tokenTypeTag, countTypeTag);
-	}
-
-	@Override
-	public IToken createToken() {
-		return new HashedUTF8WordToken(tokenTypeTag, countTypeTag);
-	}
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IBinaryTokenizer.java
deleted file mode 100644
index 05c6d0b..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IBinaryTokenizer.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-public interface IBinaryTokenizer {
-	public IToken getToken();
-
-	public boolean hasNext();
-
-	public void next();
-
-	public void reset(byte[] data, int start, int length);
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IBinaryTokenizerFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IBinaryTokenizerFactory.java
deleted file mode 100644
index bfe78ee..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IBinaryTokenizerFactory.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import java.io.Serializable;
-
-public interface IBinaryTokenizerFactory extends Serializable {
-	public IBinaryTokenizer createTokenizer();
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/INGramToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/INGramToken.java
deleted file mode 100644
index befc6d2..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/INGramToken.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-public interface INGramToken {
-	public int getNumPostChars();
-
-	public int getNumPreChars();
-
-	public void setNumPrePostChars(int numPreChars, int numPostChars);
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
deleted file mode 100644
index 47467a1..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-
-public interface IToken {
-	public byte[] getData();
-
-	public int getLength();
-
-	public int getStart();
-
-	public int getTokenLength();
-
-	public void reset(byte[] data, int start, int length, int tokenLength,
-			int tokenCount);
-
-	public void serializeToken(GrowableArray out) throws IOException;
-
-	public void serializeTokenCount(GrowableArray out) throws IOException;
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/ITokenFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/ITokenFactory.java
deleted file mode 100644
index 8b5d71d..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/ITokenFactory.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import java.io.Serializable;
-
-public interface ITokenFactory extends Serializable {
-    public IToken createToken();
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IntArray.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IntArray.java
deleted file mode 100644
index 2eb9ff4..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IntArray.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Rares Vernica <rares (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import java.util.Arrays;
-
-public class IntArray {
-    private static final int SIZE = 128;
-
-    private int[] data;
-    private int length;
-
-    public IntArray() {
-        data = new int[SIZE];
-        length = 0;
-    }
-
-    public void add(int d) {
-        if (length == data.length) {
-            data = Arrays.copyOf(data, data.length << 1);
-        }
-        data[length++] = d;
-    }
-
-    public int[] get() {
-        return data;
-    }
-
-    public int get(int i) {
-        return data[i];
-    }
-
-    public int length() {
-        return length;
-    }
-
-    public void reset() {
-        length = 0;
-    }
-
-    public void sort() {
-        sort(0, length);
-    }
-
-    public void sort(int start, int end) {
-        Arrays.sort(data, start, end);
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder out = new StringBuilder();
-        out.append('[');
-        for (int i = 0; i < length; ++i) {
-            out.append(data[i]);
-            if (i < length - 1) {
-                out.append(',');
-                out.append(' ');
-            }
-        }
-        out.append(']');
-        return out.toString();
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
deleted file mode 100644
index fdfc02f..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-
-public class NGramUTF8StringBinaryTokenizer extends AbstractUTF8StringBinaryTokenizer {
-
-    private int gramLength;
-    private boolean usePrePost;
-
-    private int gramNum;
-    private int totalGrams;
-
-    private final INGramToken concreteToken;
-
-    public NGramUTF8StringBinaryTokenizer(int gramLength, boolean usePrePost, boolean ignoreTokenCount,
-            boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
-        super(ignoreTokenCount, sourceHasTypeTag, tokenFactory);
-        this.gramLength = gramLength;
-        this.usePrePost = usePrePost;
-        concreteToken = (INGramToken) token;
-    }
-
-    @Override
-    public boolean hasNext() {
-        if (gramNum < totalGrams) {
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public void next() {
-        int currentTokenStart = index;
-        int tokenCount = 1;
-        int numPreChars = 0;
-        int numPostChars = 0;
-        if (usePrePost) {
-            numPreChars = Math.max(gramLength - gramNum - 1, 0);
-            numPostChars = (gramNum > totalGrams - gramLength) ? gramLength - totalGrams + gramNum : 0;
-        }
-        gramNum++;
-
-        concreteToken.setNumPrePostChars(numPreChars, numPostChars);
-        if (numPreChars == 0) {
-            index += UTF8StringPointable.charSize(data, index);
-        }
-
-        // compute token count
-        // ignore pre and post grams for duplicate detection
-        if (!ignoreTokenCount && numPreChars == 0 && numPostChars == 0) {
-            int tmpIndex = start;
-            while (tmpIndex < currentTokenStart) {
-                tokenCount++; // assume found
-                int offset = 0;
-                for (int j = 0; j < gramLength; j++) {
-                    if (Character.toLowerCase(UTF8StringPointable.charAt(data, currentTokenStart + offset)) != Character
-                            .toLowerCase(UTF8StringPointable.charAt(data, tmpIndex + offset))) {
-                        tokenCount--;
-                        break;
-                    }
-                    offset += UTF8StringPointable.charSize(data, tmpIndex + offset);
-                }
-                tmpIndex += UTF8StringPointable.charSize(data, tmpIndex);
-            }
-        }
-
-        // set token
-        token.reset(data, currentTokenStart, length, gramLength, tokenCount);
-    }
-
-    @Override
-    public void reset(byte[] data, int start, int length) {
-        super.reset(data, start, length);
-        gramNum = 0;
-
-        int numChars = 0;
-        int pos = index;
-        int end = pos + utf8Length;
-        while (pos < end) {
-            numChars++;
-            pos += UTF8StringPointable.charSize(data, pos);
-        }
-
-        if (usePrePost) {
-            totalGrams = numChars + gramLength - 1;
-        } else {
-            totalGrams = numChars - gramLength + 1;
-        }
-    }
-
-    public void setGramlength(int gramLength) {
-        this.gramLength = gramLength;
-    }
-
-    public void setPrePost(boolean usePrePost) {
-        this.usePrePost = usePrePost;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
deleted file mode 100644
index 8cb9818..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
-
-public class UTF8NGramToken extends AbstractUTF8Token implements INGramToken {
-
-    public final static char PRECHAR = '#';
-
-    public final static char POSTCHAR = '$';
-
-    protected int numPreChars;
-    protected int numPostChars;
-
-    public UTF8NGramToken(byte tokenTypeTag, byte countTypeTag) {
-        super(tokenTypeTag, countTypeTag);
-    }
-
-    @Override
-    public int getNumPostChars() {
-        return numPreChars;
-    }
-
-    @Override
-    public int getNumPreChars() {
-        return numPostChars;
-    }
-
-    @Override
-    public void serializeToken(GrowableArray out) throws IOException {
-        handleTokenTypeTag(out.getDataOutput());
-        int tokenUTF8LenOff = out.getLength();
-
-        // regular chars
-        int numRegChars = tokenLength - numPreChars - numPostChars;
-
-        // assuming pre and post char need 1-byte each in utf8
-        int tokenUTF8Len = numPreChars + numPostChars;
-
-        // Write dummy UTF length which will be correctly set later.
-        out.getDataOutput().writeShort(0);
-
-        // pre chars
-        for (int i = 0; i < numPreChars; i++) {
-            StringUtils.writeCharAsModifiedUTF8(PRECHAR, out.getDataOutput());
-        }
-
-        int pos = start;
-        for (int i = 0; i < numRegChars; i++) {
-            char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
-            tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
-            pos += UTF8StringPointable.charSize(data, pos);
-        }
-
-        // post chars
-        for (int i = 0; i < numPostChars; i++) {
-            StringUtils.writeCharAsModifiedUTF8(POSTCHAR, out.getDataOutput());
-        }
-
-        // Set UTF length of token.
-        out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
-        out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
-    }
-
-    public void setNumPrePostChars(int numPreChars, int numPostChars) {
-        this.numPreChars = numPreChars;
-        this.numPostChars = numPostChars;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramTokenFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramTokenFactory.java
deleted file mode 100644
index 968d8e1..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramTokenFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-public class UTF8NGramTokenFactory extends AbstractUTF8TokenFactory {
-
-	private static final long serialVersionUID = 1L;
-
-	public UTF8NGramTokenFactory() {
-		super();
-	}
-
-	public UTF8NGramTokenFactory(byte tokenTypeTag, byte countTypeTag) {
-		super(tokenTypeTag, countTypeTag);
-	}
-
-	@Override
-	public IToken createToken() {
-		return new UTF8NGramToken(tokenTypeTag, countTypeTag);
-	}
-
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
deleted file mode 100644
index 9d7fe7c..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
-
-public class UTF8WordToken extends AbstractUTF8Token {
-
-    public UTF8WordToken(byte tokenTypeTag, byte countTypeTag) {
-        super(tokenTypeTag, countTypeTag);
-    }
-
-    @Override
-    public void serializeToken(GrowableArray out) throws IOException {
-        handleTokenTypeTag(out.getDataOutput());
-        int tokenUTF8LenOff = out.getLength();
-        int tokenUTF8Len = 0;
-        // Write dummy UTF length which will be correctly set later.
-        out.getDataOutput().writeShort(0);
-        int pos = start;
-        for (int i = 0; i < tokenLength; i++) {
-            char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
-            tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
-            pos += UTF8StringPointable.charSize(data, pos);
-        }
-        // Set UTF length of token.
-        out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
-        out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
-    }
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordTokenFactory.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordTokenFactory.java
deleted file mode 100644
index 4358254..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordTokenFactory.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-
-public class UTF8WordTokenFactory extends AbstractUTF8TokenFactory {
-
-	private static final long serialVersionUID = 1L;
-
-	public UTF8WordTokenFactory() {
-		super();
-	}
-
-	public UTF8WordTokenFactory(byte tokenTypeTag, byte countTypeTag) {
-		super(tokenTypeTag, countTypeTag);
-	}
-
-	@Override
-	public IToken createToken() {
-		return new UTF8WordToken(tokenTypeTag, countTypeTag);
-	}
-
-}
diff --git a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java b/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
deleted file mode 100644
index a1d1f06..0000000
--- a/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex.util;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
-
-public class InvertedIndexUtils {
-    // Type traits to be appended to the token type trait which finally form the BTree field type traits.
-    private static final ITypeTraits[] btreeValueTypeTraits = new ITypeTraits[4];
-    static {
-        // startPageId
-        btreeValueTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
-        // endPageId
-        btreeValueTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
-        // startOff
-        btreeValueTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
-        // numElements
-        btreeValueTypeTraits[3] = IntegerPointable.TYPE_TRAITS;
-    }
-
-    public static ITypeTraits[] getBTreeTypeTraits(ITypeTraits[] tokenTypeTraits) {
-        ITypeTraits[] btreeTypeTraits = new ITypeTraits[tokenTypeTraits.length + btreeValueTypeTraits.length];
-        // Set key type traits.
-        for (int i = 0; i < tokenTypeTraits.length; i++) {
-            btreeTypeTraits[i] = tokenTypeTraits[i];
-        }
-        // Set value type traits.
-        for (int i = 0; i < btreeValueTypeTraits.length; i++) {
-            btreeTypeTraits[i + tokenTypeTraits.length] = btreeValueTypeTraits[i];
-        }
-        return btreeTypeTraits;
-    }
-}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/pom.xml b/hyracks/hyracks-storage-am-lsm-btree/pom.xml
new file mode 100644
index 0000000..8fa09d0
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/pom.xml
@@ -0,0 +1,47 @@
+<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>
+  <artifactId>hyracks-storage-am-lsm-btree</artifactId>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>    
+    <dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-btree</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency> 
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-bloomfilter</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency> 
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-lsm-common</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>  	
+  </dependencies>
+</project>
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
new file mode 100644
index 0000000..06f06c6
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+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.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+
+public class LSMBTreeDataflowHelper extends AbstractLSMIndexDataflowHelper {
+
+    public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, mergePolicy, opTrackerFactory,
+                ioScheduler, ioOpCallbackProvider);
+    }
+
+    public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
+    }
+
+    @Override
+    public ITreeIndex createIndexInstance() throws HyracksDataException {
+        AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
+        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+        IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
+                memNumPages, new TransientFileMapManager());
+        IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(memNumPages, metaDataFrameFactory);
+        return LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ctx.getIOManager(), file, opDesc
+                .getStorageManager().getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx),
+                treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc
+                        .getTreeIndexBloomFilterKeyFields(), mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider, partition);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
new file mode 100644
index 0000000..ebf4bc0
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
+
+public class LSMBTreeDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public LSMBTreeDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+        super(mergePolicyProvider, opTrackerFactory, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
+                memNumPages);
+    }
+
+    @Override
+    public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition) {
+        return new LSMBTreeDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
+                mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
+                ioOpCallbackProvider);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/AntimatterAwareTupleAcceptor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/AntimatterAwareTupleAcceptor.java
new file mode 100644
index 0000000..e88aea6
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/AntimatterAwareTupleAcceptor.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.api.ITupleAcceptor;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
+
+public enum AntimatterAwareTupleAcceptor implements ITupleAcceptor {
+    INSTANCE;
+
+    @Override
+    public boolean accept(ITupleReference tuple) {
+        if (tuple == null) {
+            return true;
+        }
+        return ((LSMBTreeTupleReference) tuple).isAntimatter();
+    }
+
+}
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
new file mode 100644
index 0000000..655cd12
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -0,0 +1,640 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import java.io.File;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeBulkLoader;
+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.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
+
+    // In-memory components.   
+    private final LSMBTreeMutableComponent mutableComponent;
+
+    // For creating BTree's used in flush and merge.
+    private final LSMBTreeImmutableComponentFactory componentFactory;
+    // For creating BTree's used in bulk load. Different from diskBTreeFactory
+    // because it should have a different tuple writer in it's leaf frames.
+    private final LSMBTreeImmutableComponentFactory bulkLoadComponentFactory;
+
+    // Common for in-memory and on-disk components.
+    private final ITreeIndexFrameFactory insertLeafFrameFactory;
+    private final ITreeIndexFrameFactory deleteLeafFrameFactory;
+    private final IBinaryComparatorFactory[] cmpFactories;
+
+    public LSMBTree(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
+            ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager,
+            TreeIndexFactory<BTree> diskBTreeFactory, TreeIndexFactory<BTree> bulkLoadBTreeFactory,
+            BloomFilterFactory bloomFilterFactory, IFileMapProvider diskFileMapProvider, int fieldCount,
+            IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(memFreePageManager, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        mutableComponent = new LSMBTreeMutableComponent(new BTree(memBufferCache,
+                ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), memFreePageManager, interiorFrameFactory,
+                insertLeafFrameFactory, cmpFactories, fieldCount, new FileReference(new File("membtree"))),
+                memFreePageManager);
+        this.insertLeafFrameFactory = insertLeafFrameFactory;
+        this.deleteLeafFrameFactory = deleteLeafFrameFactory;
+        this.cmpFactories = cmpFactories;
+        componentFactory = new LSMBTreeImmutableComponentFactory(diskBTreeFactory, bloomFilterFactory);
+        bulkLoadComponentFactory = new LSMBTreeImmutableComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory);
+    }
+
+    @Override
+    public synchronized void create() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to create the index since it is activated.");
+        }
+
+        fileManager.deleteDirs();
+        fileManager.createDirs();
+        componentsRef.get().clear();
+    }
+
+    @Override
+    public synchronized void activate() throws HyracksDataException {
+        if (isActivated) {
+            return;
+        }
+
+        ((InMemoryBufferCache) mutableComponent.getBTree().getBufferCache()).open();
+        mutableComponent.getBTree().create();
+        mutableComponent.getBTree().activate();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        immutableComponents.clear();
+        List<LSMComponentFileReferences> validFileReferences;
+        try {
+            validFileReferences = fileManager.cleanupAndGetValidFiles();
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+        for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
+            LSMBTreeImmutableComponent component;
+            try {
+                component = createDiskComponent(componentFactory,
+                        lsmComonentFileReference.getInsertIndexFileReference(),
+                        lsmComonentFileReference.getBloomFilterFileReference(), false);
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+            immutableComponents.add(component);
+        }
+        isActivated = true;
+    }
+
+    @Override
+    public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
+        if (!isActivated) {
+            return;
+        }
+
+        if (flushOnExit) {
+            BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
+                    ioOpCallbackProvider.getIOOperationCallback(this));
+            ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            accessor.scheduleFlush(cb);
+            try {
+                cb.waitForIO();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) c;
+            BTree btree = component.getBTree();
+            BloomFilter bloomFilter = component.getBloomFilter();
+            btree.deactivate();
+            bloomFilter.deactivate();
+        }
+        mutableComponent.getBTree().deactivate();
+        mutableComponent.getBTree().destroy();
+        ((InMemoryBufferCache) mutableComponent.getBTree().getBufferCache()).close();
+        isActivated = false;
+    }
+
+    @Override
+    public synchronized void deactivate() throws HyracksDataException {
+        deactivate(true);
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to destroy the index since it is activated.");
+        }
+
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) c;
+            component.getBTree().destroy();
+            component.getBloomFilter().destroy();
+        }
+        mutableComponent.getBTree().destroy();
+        fileManager.deleteDirs();
+    }
+
+    @Override
+    public void clear() throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("Failed to clear the index since it is not activated.");
+        }
+
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        mutableComponent.getBTree().clear();
+        for (ILSMComponent c : immutableComponents) {
+            LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) c;
+            component.getBloomFilter().deactivate();
+            component.getBTree().deactivate();
+            component.getBloomFilter().destroy();
+            component.getBTree().destroy();
+        }
+        immutableComponents.clear();
+    }
+
+    @Override
+    public void getOperationalComponents(ILSMIndexOperationContext ctx) {
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
+        operationalComponents.clear();
+        switch (ctx.getOperation()) {
+            case UPDATE:
+            case UPSERT:
+            case PHYSICALDELETE:
+            case FLUSH:
+            case DELETE:
+                operationalComponents.add(mutableComponent);
+                break;
+            case SEARCH:
+            case INSERT:
+                operationalComponents.add(mutableComponent);
+                operationalComponents.addAll(immutableComponents);
+                break;
+            case MERGE:
+                operationalComponents.addAll(immutableComponents);
+                break;
+            default:
+                throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
+        }
+    }
+
+    @Override
+    public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException {
+        LSMBTreeOpContext ctx = (LSMBTreeOpContext) ictx;
+        switch (ctx.getOperation()) {
+            case PHYSICALDELETE:
+                ctx.memBTreeAccessor.delete(tuple);
+                break;
+            case INSERT:
+                insert(tuple, ctx);
+                break;
+            default:
+                ctx.memBTreeAccessor.upsert(tuple);
+                break;
+        }
+    }
+
+    private boolean insert(ITupleReference tuple, LSMBTreeOpContext ctx) throws HyracksDataException, IndexException {
+        MultiComparator comparator = MultiComparator.createIgnoreFieldLength(mutableComponent.getBTree()
+                .getComparatorFactories());
+        LSMBTreePointSearchCursor searchCursor = new LSMBTreePointSearchCursor(ctx);
+        IIndexCursor memCursor = new BTreeRangeSearchCursor(ctx.memBTreeOpCtx.leafFrame, false);
+        RangePredicate predicate = new RangePredicate(tuple, tuple, true, true, comparator, comparator);
+
+        // first check the inmemory component
+        ctx.memBTreeAccessor.search(memCursor, predicate);
+        try {
+            if (memCursor.hasNext()) {
+                memCursor.next();
+                LSMBTreeTupleReference lsmbtreeTuple = (LSMBTreeTupleReference) memCursor.getTuple();
+                if (!lsmbtreeTuple.isAntimatter()) {
+                    throw new BTreeDuplicateKeyException("Failed to insert key since key already exists.");
+                } else {
+                    memCursor.close();
+                    ctx.memBTreeAccessor.upsertIfConditionElseInsert(tuple, AntimatterAwareTupleAcceptor.INSTANCE);
+                    return true;
+                }
+            }
+        } finally {
+            memCursor.close();
+        }
+
+        // TODO: Can we just remove the above code that search the mutable component and do it together with the search call below? i.e. instead of passing false to the lsmHarness.search(), we pass true to include the mutable component?
+        // the key was not in the inmemory component, so check the disk components
+        search(ctx, searchCursor, predicate);
+        try {
+            if (searchCursor.hasNext()) {
+                throw new BTreeDuplicateKeyException("Failed to insert key since key already exists.");
+            }
+        } finally {
+            searchCursor.close();
+        }
+        ctx.memBTreeAccessor.upsertIfConditionElseInsert(tuple, AntimatterAwareTupleAcceptor.INSTANCE);
+
+        return true;
+    }
+
+    @Override
+    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException {
+        LSMBTreeOpContext ctx = (LSMBTreeOpContext) ictx;
+        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
+        int numBTrees = operationalComponents.size();
+        assert numBTrees > 0;
+
+        boolean includeMutableComponent = operationalComponents.get(0) == mutableComponent;
+        LSMBTreeCursorInitialState initialState = new LSMBTreeCursorInitialState(numBTrees, insertLeafFrameFactory,
+                ctx.cmp, ctx.bloomFilterCmp, includeMutableComponent, lsmHarness, ctx.memBTreeAccessor, pred,
+                ctx.searchCallback, operationalComponents);
+        cursor.open(initialState, pred);
+    }
+
+    @Override
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException {
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        LSMBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        assert ctx.getComponentHolder().size() == 1;
+        ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
+        opCtx.setOperation(IndexOperation.FLUSH);
+        opCtx.getComponentHolder().add(flushingComponent);
+        ILSMIndexAccessorInternal flushAccessor = new LSMBTreeAccessor(lsmHarness, opCtx);
+        ioScheduler.scheduleOperation(new LSMBTreeFlushOperation(flushAccessor, flushingComponent, componentFileRefs
+                .getInsertIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), callback));
+    }
+
+    @Override
+    public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        LSMBTreeFlushOperation flushOp = (LSMBTreeFlushOperation) operation;
+        LSMBTreeMutableComponent flushingComponent = (LSMBTreeMutableComponent) flushOp.getFlushingComponent();
+        IIndexAccessor accessor = flushingComponent.getBTree().createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+
+        RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
+        IIndexCursor countingCursor = ((BTreeAccessor) accessor).createCountingSearchCursor();
+        accessor.search(countingCursor, nullPred);
+        long numElements = 0L;
+        try {
+            while (countingCursor.hasNext()) {
+                countingCursor.next();
+                ITupleReference countTuple = countingCursor.getTuple();
+                numElements = IntegerSerializerDeserializer.getInt(countTuple.getFieldData(0),
+                        countTuple.getFieldStart(0));
+            }
+        } finally {
+            countingCursor.close();
+        }
+
+        int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
+        BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+
+        LSMBTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getBTreeFlushTarget(),
+                flushOp.getBloomFilterFlushTarget(), true);
+        IIndexBulkLoader bulkLoader = component.getBTree().createBulkLoader(1.0f, false, numElements);
+        IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numElements,
+                bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+
+        IIndexCursor scanCursor = accessor.createSearchCursor();
+        accessor.search(scanCursor, nullPred);
+        try {
+            while (scanCursor.hasNext()) {
+                scanCursor.next();
+                builder.add(scanCursor.getTuple());
+                bulkLoader.add(scanCursor.getTuple());
+            }
+        } finally {
+            scanCursor.close();
+            builder.end();
+        }
+        bulkLoader.end();
+        return component;
+    }
+
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        LSMBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+        opCtx.getComponentHolder().addAll(mergingComponents);
+        ITreeIndexCursor cursor = new LSMBTreeRangeSearchCursor(opCtx);
+        RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+        search(opCtx, cursor, rangePred);
+
+        opCtx.setOperation(IndexOperation.MERGE);
+        BTree firstBTree = (BTree) ((LSMBTreeImmutableComponent) mergingComponents.get(0)).getBTree();
+        BTree lastBTree = (BTree) ((LSMBTreeImmutableComponent) mergingComponents.get(mergingComponents.size() - 1))
+                .getBTree();
+        FileReference firstFile = diskFileMapProvider.lookupFileName(firstBTree.getFileId());
+        FileReference lastFile = diskFileMapProvider.lookupFileName(lastBTree.getFileId());
+        LSMComponentFileReferences relMergeFileRefs = fileManager.getRelMergeFileReference(firstFile.getFile()
+                .getName(), lastFile.getFile().getName());
+        ILSMIndexAccessorInternal accessor = new LSMBTreeAccessor(lsmHarness, opCtx);
+        ioScheduler.scheduleOperation(new LSMBTreeMergeOperation(accessor, mergingComponents, cursor, relMergeFileRefs
+                .getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback));
+    }
+
+    @Override
+    public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
+            throws HyracksDataException, IndexException {
+        LSMBTreeMergeOperation mergeOp = (LSMBTreeMergeOperation) operation;
+        ITreeIndexCursor cursor = mergeOp.getCursor();
+        mergedComponents.addAll(mergeOp.getMergingComponents());
+
+        long numElements = 0L;
+        for (int i = 0; i < mergedComponents.size(); ++i) {
+            numElements += ((LSMBTreeImmutableComponent) mergedComponents.get(i)).getBloomFilter().getNumElements();
+        }
+
+        int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
+        BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+        LSMBTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
+                mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
+
+        IIndexBulkLoader bulkLoader = mergedComponent.getBTree().createBulkLoader(1.0f, false, numElements);
+        IIndexBulkLoader builder = mergedComponent.getBloomFilter().createBuilder(numElements,
+                bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+                ITupleReference frameTuple = cursor.getTuple();
+                builder.add(frameTuple);
+                bulkLoader.add(frameTuple);
+            }
+        } finally {
+            cursor.close();
+            builder.end();
+        }
+        bulkLoader.end();
+        return mergedComponent;
+    }
+
+    private LSMBTreeImmutableComponent createDiskComponent(LSMBTreeImmutableComponentFactory factory,
+            FileReference btreeFileRef, FileReference bloomFilterFileRef, boolean createComponent)
+            throws HyracksDataException, IndexException {
+        // Create new BTree instance.
+        LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) factory
+                .createLSMComponentInstance(new LSMComponentFileReferences(btreeFileRef, null, bloomFilterFileRef));
+        if (createComponent) {
+            component.getBTree().create();
+            component.getBloomFilter().create();
+        }
+        // BTree will be closed during cleanup of merge().
+        component.getBTree().activate();
+        component.getBloomFilter().activate();
+        return component;
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
+            throws TreeIndexException {
+        try {
+            return new LSMBTreeBulkLoader(fillLevel, verifyInput, numElementsHint);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    private ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        return createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+                componentFileRefs.getBloomFilterFileReference(), true);
+    }
+
+    @Override
+    public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException {
+        // The order of forcing the dirty page to be flushed is critical. The bloom filter must be always done first.
+        LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) lsmComponent;
+        // Flush the bloom filter first.
+        int fileId = component.getBloomFilter().getFileId();
+        IBufferCache bufferCache = component.getBTree().getBufferCache();
+        int startPage = 0;
+        int maxPage = component.getBloomFilter().getNumPages();
+        forceFlushDirtyPages(bufferCache, fileId, startPage, maxPage);
+        forceFlushDirtyPages(component.getBTree());
+        markAsValidInternal(component.getBTree());
+    }
+
+    public class LSMBTreeBulkLoader implements IIndexBulkLoader {
+        private final ILSMComponent component;
+        private final BTreeBulkLoader bulkLoader;
+        private final IIndexBulkLoader builder;
+        private boolean endHasBeenCalled = false;
+
+        public LSMBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+                throws TreeIndexException, HyracksDataException {
+            try {
+                component = createBulkLoadTarget();
+            } catch (HyracksDataException e) {
+                throw new TreeIndexException(e);
+            } catch (IndexException e) {
+                throw new TreeIndexException(e);
+            }
+            bulkLoader = (BTreeBulkLoader) ((LSMBTreeImmutableComponent) component).getBTree().createBulkLoader(
+                    fillFactor, verifyInput, numElementsHint);
+
+            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
+            BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                    MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+            builder = ((LSMBTreeImmutableComponent) component).getBloomFilter().createBuilder(numElementsHint,
+                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
+            try {
+                bulkLoader.add(tuple);
+                builder.add(tuple);
+            } catch (IndexException e) {
+                handleException();
+                throw e;
+            } catch (HyracksDataException e) {
+                handleException();
+                throw e;
+            } catch (RuntimeException e) {
+                handleException();
+                throw e;
+            }
+        }
+
+        protected void handleException() throws HyracksDataException, IndexException {
+            if (!endHasBeenCalled) {
+                builder.end();
+            }
+            ((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);
+        }
+
+    }
+
+    public LSMBTreeOpContext createOpContext(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new LSMBTreeOpContext(mutableComponent.getBTree(), insertLeafFrameFactory, deleteLeafFrameFactory,
+                modificationCallback, searchCallback, componentFactory.getBloomFilterKeyFields().length);
+    }
+
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new LSMBTreeAccessor(lsmHarness, createOpContext(modificationCallback, searchCallback));
+    }
+
+    public class LSMBTreeAccessor extends LSMTreeIndexAccessor {
+        public LSMBTreeAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx) {
+            super(lsmHarness, ctx);
+        }
+
+        @Override
+        public IIndexCursor createSearchCursor() {
+            return new LSMBTreeSearchCursor(ctx);
+        }
+
+        public MultiComparator getMultiComparator() {
+            LSMBTreeOpContext concreteCtx = (LSMBTreeOpContext) ctx;
+            return concreteCtx.cmp;
+        }
+    }
+
+    @Override
+    public IBufferCache getBufferCache() {
+        return diskBufferCache;
+    }
+
+    public IBinaryComparatorFactory[] getComparatorFactories() {
+        return cmpFactories;
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getInteriorFrameFactory() {
+        return mutableComponent.getBTree().getInteriorFrameFactory();
+    }
+
+    @Override
+    public int getFieldCount() {
+        return mutableComponent.getBTree().getFieldCount();
+    }
+
+    @Override
+    public int getFileId() {
+        return mutableComponent.getBTree().getFileId();
+    }
+
+    @Override
+    public IFreePageManager getFreePageManager() {
+        return mutableComponent.getBTree().getFreePageManager();
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getLeafFrameFactory() {
+        return mutableComponent.getBTree().getLeafFrameFactory();
+    }
+
+    @Override
+    public long getMemoryAllocationSize() {
+        InMemoryBufferCache memBufferCache = (InMemoryBufferCache) mutableComponent.getBTree().getBufferCache();
+        return memBufferCache.getNumPages() * memBufferCache.getPageSize();
+    }
+
+    @Override
+    public int getRootPageId() {
+        return mutableComponent.getBTree().getRootPageId();
+    }
+
+    public boolean isEmptyIndex() throws HyracksDataException {
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        return immutableComponents.isEmpty()
+                && mutableComponent.getBTree().isEmptyTree(
+                        mutableComponent.getBTree().getInteriorFrameFactory().createFrame());
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        mutableComponent.getBTree().validate();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            BTree btree = (BTree) ((LSMBTreeImmutableComponent) c).getBTree();
+            btree.validate();
+		}
+	}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java
new file mode 100644
index 0000000..2b7029b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public class LSMBTreeCursorInitialState implements ICursorInitialState {
+
+    private final int numBTrees;
+    private final ITreeIndexFrameFactory leafFrameFactory;
+    private MultiComparator cmp;
+    private final MultiComparator bloomFilterCmp;
+    private final boolean includeMemComponent;
+    private final ILSMHarness lsmHarness;
+
+    private final IIndexAccessor memBtreeAccessor;
+    private final ISearchPredicate predicate;
+    private ISearchOperationCallback searchCallback;
+
+    private final List<ILSMComponent> operationalComponents;
+
+    public LSMBTreeCursorInitialState(int numBTrees, ITreeIndexFrameFactory leafFrameFactory, MultiComparator cmp,
+            MultiComparator bloomFilterCmp, boolean includeMemComponent, ILSMHarness lsmHarness,
+            IIndexAccessor memBtreeAccessor, ISearchPredicate predicate, ISearchOperationCallback searchCallback,
+            List<ILSMComponent> operationalComponents) {
+        this.numBTrees = numBTrees;
+        this.leafFrameFactory = leafFrameFactory;
+        this.cmp = cmp;
+        this.bloomFilterCmp = bloomFilterCmp;
+        this.includeMemComponent = includeMemComponent;
+        this.lsmHarness = lsmHarness;
+        this.searchCallback = searchCallback;
+        this.memBtreeAccessor = memBtreeAccessor;
+        this.predicate = predicate;
+        this.operationalComponents = operationalComponents;
+    }
+
+    public int getNumBTrees() {
+        return numBTrees;
+    }
+
+    public ITreeIndexFrameFactory getLeafFrameFactory() {
+        return leafFrameFactory;
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        return null;
+    }
+
+    @Override
+    public void setPage(ICachedPage page) {
+    }
+
+    public boolean getIncludeMemComponent() {
+        return includeMemComponent;
+    }
+
+    public ILSMHarness getLSMHarness() {
+        return lsmHarness;
+    }
+
+    @Override
+    public ISearchOperationCallback getSearchOperationCallback() {
+        return searchCallback;
+    }
+
+    @Override
+    public void setSearchOperationCallback(ISearchOperationCallback searchCallback) {
+        this.searchCallback = searchCallback;
+    }
+
+    public List<ILSMComponent> getOperationalComponents() {
+        return operationalComponents;
+    }
+
+    public IIndexAccessor getMemBTreeAccessor() {
+        return memBtreeAccessor;
+    }
+
+    public ISearchPredicate getSearchPredicate() {
+        return predicate;
+    }
+
+    public MultiComparator getBloomFilterComparator() {
+        return bloomFilterCmp;
+    }
+
+    @Override
+    public MultiComparator getOriginalKeyComparator() {
+        return cmp;
+    }
+
+    @Override
+    public void setOriginialKeyComparator(MultiComparator originalCmp) {
+        this.cmp = originalCmp;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
new file mode 100644
index 0000000..38766c3
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
@@ -0,0 +1,179 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.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;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMBTreeFileManager extends AbstractLSMIndexFileManager {
+    private static final String BTREE_STRING = "b";
+
+    private final TreeIndexFactory<? extends ITreeIndex> btreeFactory;
+
+    public LSMBTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> btreeFactory, int startIODeviceIndex) {
+        super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+        this.btreeFactory = btreeFactory;
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelFlushFileReference() {
+        Date date = new Date();
+        String ts = formatter.format(date);
+        String baseName = baseDir + ts + SPLIT_STRING + ts;
+        // Begin timestamp and end timestamp are identical since it is a flush
+        return new LSMComponentFileReferences(createFlushFile(baseName + SPLIT_STRING + BTREE_STRING), null,
+                createFlushFile(baseName + SPLIT_STRING + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
+            throws HyracksDataException {
+        String[] firstTimestampRange = firstFileName.split(SPLIT_STRING);
+        String[] lastTimestampRange = lastFileName.split(SPLIT_STRING);
+
+        String baseName = baseDir + firstTimestampRange[0] + SPLIT_STRING + lastTimestampRange[1];
+        // Get the range of timestamps by taking the earliest and the latest timestamps
+        return new LSMComponentFileReferences(createMergeFile(baseName + SPLIT_STRING + BTREE_STRING), null,
+                createMergeFile(baseName + SPLIT_STRING + BLOOM_FILTER_STRING));
+    }
+
+    private static FilenameFilter btreeFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".") && name.endsWith(BTREE_STRING);
+        }
+    };
+
+    @Override
+    public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException, IndexException {
+        List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
+        ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
+        ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
+
+        // Gather files from all IODeviceHandles.
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            cleanupAndGetValidFilesInternal(dev, bloomFilterFilter, null, allBloomFilterFiles);
+            HashSet<String> bloomFilterFilesSet = new HashSet<String>();
+            for (ComparableFileName cmpFileName : allBloomFilterFiles) {
+                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+                bloomFilterFilesSet.add(cmpFileName.fileName.substring(0, index));
+            }
+            // List of valid BTree files that may or may not have a bloom filter buddy. Will check for buddies below.
+            ArrayList<ComparableFileName> tmpAllBTreeFiles = new ArrayList<ComparableFileName>();
+            cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, tmpAllBTreeFiles);
+            // Look for buddy bloom filters for all valid BTrees. 
+            // If no buddy is found, delete the file, otherwise add the BTree to allBTreeFiles. 
+            for (ComparableFileName cmpFileName : tmpAllBTreeFiles) {
+                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+                String file = cmpFileName.fileName.substring(0, index);
+                if (bloomFilterFilesSet.contains(file)) {
+                    allBTreeFiles.add(cmpFileName);
+                } else {
+                    // Couldn't find the corresponding bloom filter file; thus, delete
+                    // the BTree file.
+                    File invalidBTreeFile = new File(cmpFileName.fullPath);
+                    invalidBTreeFile.delete();
+                }
+            }
+        }
+        // Sanity check.
+        if (allBTreeFiles.size() != allBloomFilterFiles.size()) {
+            throw new HyracksDataException(
+                    "Unequal number of valid BTree and bloom filter files found. Aborting cleanup.");
+        }
+
+        // Trivial cases.
+        if (allBTreeFiles.isEmpty() || allBloomFilterFiles.isEmpty()) {
+            return validFiles;
+        }
+
+        if (allBTreeFiles.size() == 1 && allBloomFilterFiles.size() == 1) {
+            validFiles.add(new LSMComponentFileReferences(allBTreeFiles.get(0).fileRef, null, allBloomFilterFiles
+                    .get(0).fileRef));
+            return validFiles;
+        }
+
+        // Sorts files names from earliest to latest timestamp.
+        Collections.sort(allBTreeFiles);
+        Collections.sort(allBloomFilterFiles);
+
+        List<ComparableFileName> validComparableBTreeFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastBTree = allBTreeFiles.get(0);
+        validComparableBTreeFiles.add(lastBTree);
+
+        List<ComparableFileName> validComparableBloomFilterFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastBloomFilter = allBloomFilterFiles.get(0);
+        validComparableBloomFilterFiles.add(lastBloomFilter);
+
+        for (int i = 1; i < allBTreeFiles.size(); i++) {
+            ComparableFileName currentBTree = allBTreeFiles.get(i);
+            ComparableFileName currentBloomFilter = allBloomFilterFiles.get(i);
+            // Current start timestamp is greater than last stop timestamp.
+            if (currentBTree.interval[0].compareTo(lastBTree.interval[1]) > 0
+                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0) {
+                validComparableBTreeFiles.add(currentBTree);
+                validComparableBloomFilterFiles.add(currentBloomFilter);
+                lastBTree = currentBTree;
+                lastBloomFilter = currentBloomFilter;
+            } else if (currentBTree.interval[0].compareTo(lastBTree.interval[0]) >= 0
+                    && currentBTree.interval[1].compareTo(lastBTree.interval[1]) <= 0
+                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[0]) >= 0
+                    && currentBloomFilter.interval[1].compareTo(lastBloomFilter.interval[1]) <= 0) {
+                // Invalid files are completely contained in last interval.
+                File invalidBTreeFile = new File(currentBTree.fullPath);
+                invalidBTreeFile.delete();
+                File invalidBloomFilterFile = new File(currentBloomFilter.fullPath);
+                invalidBloomFilterFile.delete();
+            } else {
+                // This scenario should not be possible.
+                throw new HyracksDataException("Found LSM files with overlapping but not contained timetamp intervals.");
+            }
+        }
+
+        // Sort valid files in reverse lexicographical order, such that newer
+        // files come first.
+        Collections.sort(validComparableBTreeFiles, recencyCmp);
+        Collections.sort(validComparableBloomFilterFiles, recencyCmp);
+
+        Iterator<ComparableFileName> btreeFileIter = validComparableBTreeFiles.iterator();
+        Iterator<ComparableFileName> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
+        while (btreeFileIter.hasNext() && bloomFilterFileIter.hasNext()) {
+            ComparableFileName cmpBTreeFileName = btreeFileIter.next();
+            ComparableFileName cmpBloomFilterFileName = bloomFilterFileIter.next();
+            validFiles.add(new LSMComponentFileReferences(cmpBTreeFileName.fileRef, null,
+                    cmpBloomFilterFileName.fileRef));
+        }
+
+        return validFiles;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
new file mode 100644
index 0000000..dfda07b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
@@ -0,0 +1,71 @@
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+
+public class LSMBTreeFlushOperation implements ILSMIOOperation {
+
+    private final ILSMIndexAccessorInternal accessor;
+    private final ILSMComponent flushingComponent;
+    private final FileReference btreeFlushTarget;
+    private final FileReference bloomFilterFlushTarget;
+    private final ILSMIOOperationCallback callback;
+
+    public LSMBTreeFlushOperation(ILSMIndexAccessorInternal accessor, ILSMComponent flushingComponent,
+            FileReference btreeFlushTarget, FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback) {
+        this.accessor = accessor;
+        this.flushingComponent = flushingComponent;
+        this.btreeFlushTarget = btreeFlushTarget;
+        this.bloomFilterFlushTarget = bloomFilterFlushTarget;
+        this.callback = callback;
+    }
+
+    @Override
+    public Set<IODeviceHandle> getReadDevices() {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public Set<IODeviceHandle> getWriteDevices() {
+        Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+        devs.add(btreeFlushTarget.getDeviceHandle());
+        devs.add(bloomFilterFlushTarget.getDeviceHandle());
+        return devs;
+    }
+
+    @Override
+    public void perform() throws HyracksDataException, IndexException {
+        accessor.flush(this);
+    }
+
+    @Override
+    public ILSMIOOperationCallback getCallback() {
+        return callback;
+    }
+
+    public FileReference getBTreeFlushTarget() {
+        return btreeFlushTarget;
+    }
+
+    public FileReference getBloomFilterFlushTarget() {
+        return bloomFilterFlushTarget;
+    }
+
+    public ILSMIndexAccessorInternal getAccessor() {
+        return accessor;
+    }
+
+    public ILSMComponent getFlushingComponent() {
+        return flushingComponent;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponent.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponent.java
new file mode 100644
index 0000000..daa86d9
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponent.java
@@ -0,0 +1,32 @@
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractImmutableLSMComponent;
+
+public class LSMBTreeImmutableComponent extends AbstractImmutableLSMComponent {
+    private final BTree btree;
+    private final BloomFilter bloomFilter;
+
+    public LSMBTreeImmutableComponent(BTree btree, BloomFilter bloomFilter) {
+        this.btree = btree;
+        this.bloomFilter = bloomFilter;
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        btree.deactivate();
+        btree.destroy();
+        bloomFilter.deactivate();
+        bloomFilter.destroy();
+    }
+
+    public BTree getBTree() {
+        return btree;
+    }
+
+    public BloomFilter getBloomFilter() {
+        return bloomFilter;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponentFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponentFactory.java
new file mode 100644
index 0000000..e9da5a5
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeImmutableComponentFactory.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public class LSMBTreeImmutableComponentFactory implements ILSMComponentFactory {
+    private final TreeIndexFactory<BTree> btreeFactory;
+    private final BloomFilterFactory bloomFilterFactory;
+
+    public LSMBTreeImmutableComponentFactory(TreeIndexFactory<BTree> btreeFactory, BloomFilterFactory bloomFilterFactory) {
+        this.btreeFactory = btreeFactory;
+        this.bloomFilterFactory = bloomFilterFactory;
+    }
+
+    @Override
+    public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) throws IndexException,
+            HyracksDataException {
+        return new LSMBTreeImmutableComponent(btreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+                bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()));
+    }
+
+    @Override
+    public IBufferCache getBufferCache() {
+        return btreeFactory.getBufferCache();
+    }
+
+    public int[] getBloomFilterKeyFields() {
+        return bloomFilterFactory.getBloomFilterKeyFields();
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java
new file mode 100644
index 0000000..180fb9a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+
+public class LSMBTreeMergeOperation implements ILSMIOOperation {
+
+    private final ILSMIndexAccessorInternal accessor;
+    private final List<ILSMComponent> mergingComponents;
+    private final ITreeIndexCursor cursor;
+    private final FileReference btreeMergeTarget;
+    private final FileReference bloomFilterMergeTarget;
+    private final ILSMIOOperationCallback callback;
+
+    public LSMBTreeMergeOperation(ILSMIndexAccessorInternal accessor, List<ILSMComponent> mergingComponents,
+            ITreeIndexCursor cursor, FileReference btreeMergeTarget, FileReference bloomFilterMergeTarget,
+            ILSMIOOperationCallback callback) {
+        this.accessor = accessor;
+        this.mergingComponents = mergingComponents;
+        this.cursor = cursor;
+        this.btreeMergeTarget = btreeMergeTarget;
+        this.bloomFilterMergeTarget = bloomFilterMergeTarget;
+        this.callback = callback;
+    }
+
+    @Override
+    public Set<IODeviceHandle> getReadDevices() {
+        Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+        for (ILSMComponent o : mergingComponents) {
+            LSMBTreeImmutableComponent component = (LSMBTreeImmutableComponent) o;
+            devs.add(component.getBTree().getFileReference().getDeviceHandle());
+            devs.add(component.getBloomFilter().getFileReference().getDeviceHandle());
+        }
+        return devs;
+    }
+
+    @Override
+    public Set<IODeviceHandle> getWriteDevices() {
+        Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+        devs.add(btreeMergeTarget.getDeviceHandle());
+        devs.add(bloomFilterMergeTarget.getDeviceHandle());
+        return devs;
+    }
+
+    @Override
+    public void perform() throws HyracksDataException, IndexException {
+        accessor.merge(this);
+    }
+
+    @Override
+    public ILSMIOOperationCallback getCallback() {
+        return callback;
+    }
+
+    public FileReference getBTreeMergeTarget() {
+        return btreeMergeTarget;
+    }
+
+    public FileReference getBloomFilterMergeTarget() {
+        return bloomFilterMergeTarget;
+    }
+
+    public ITreeIndexCursor getCursor() {
+        return cursor;
+    }
+
+    public List<ILSMComponent> getMergingComponents() {
+        return mergingComponents;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMutableComponent.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMutableComponent.java
new file mode 100644
index 0000000..30e79b4
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMutableComponent.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMutableLSMComponent;
+
+public class LSMBTreeMutableComponent extends AbstractMutableLSMComponent {
+
+    private final BTree btree;
+    private final IInMemoryFreePageManager mfpm;
+
+    public LSMBTreeMutableComponent(BTree btree, IInMemoryFreePageManager mfpm) {
+        this.btree = btree;
+        this.mfpm = mfpm;
+    }
+
+    public BTree getBTree() {
+        return btree;
+    }
+
+    @Override
+    protected boolean isFull() {
+        return mfpm.isFull();
+    }
+
+    @Override
+    protected void reset() throws HyracksDataException {
+        btree.clear();
+    }
+
+}
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
new file mode 100644
index 0000000..9400d2d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -0,0 +1,150 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public final class LSMBTreeOpContext implements ILSMIndexOperationContext {
+
+    public ITreeIndexFrameFactory insertLeafFrameFactory;
+    public ITreeIndexFrameFactory deleteLeafFrameFactory;
+    public IBTreeLeafFrame insertLeafFrame;
+    public IBTreeLeafFrame deleteLeafFrame;
+    public final BTree memBTree;
+    public BTree.BTreeAccessor memBTreeAccessor;
+    public BTreeOpContext memBTreeOpCtx;
+    public IndexOperation op;
+    public final MultiComparator cmp;
+    public final MultiComparator bloomFilterCmp;
+    public final IModificationOperationCallback modificationCallback;
+    public final ISearchOperationCallback searchCallback;
+    private final List<ILSMComponent> componentHolder;
+
+    public LSMBTreeOpContext(BTree memBTree, ITreeIndexFrameFactory insertLeafFrameFactory,
+            ITreeIndexFrameFactory deleteLeafFrameFactory, IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback, int numBloomFilterKeyFields) {
+        IBinaryComparatorFactory cmpFactories[] = memBTree.getComparatorFactories();
+        if (cmpFactories[0] != null) {
+            this.cmp = MultiComparator.createIgnoreFieldLength(memBTree.getComparatorFactories());
+        } else {
+            this.cmp = null;
+        }
+
+        bloomFilterCmp = MultiComparator.createIgnoreFieldLength(memBTree.getComparatorFactories(), 0,
+                numBloomFilterKeyFields);
+
+        this.memBTree = memBTree;
+        this.insertLeafFrameFactory = insertLeafFrameFactory;
+        this.deleteLeafFrameFactory = deleteLeafFrameFactory;
+        this.insertLeafFrame = (IBTreeLeafFrame) insertLeafFrameFactory.createFrame();
+        this.deleteLeafFrame = (IBTreeLeafFrame) deleteLeafFrameFactory.createFrame();
+        if (insertLeafFrame != null && this.cmp != null) {
+            insertLeafFrame.setMultiComparator(cmp);
+        }
+        if (deleteLeafFrame != null && this.cmp != null) {
+            deleteLeafFrame.setMultiComparator(cmp);
+        }
+        this.componentHolder = new LinkedList<ILSMComponent>();
+        this.modificationCallback = modificationCallback;
+        this.searchCallback = searchCallback;
+    }
+
+    @Override
+    public void setOperation(IndexOperation newOp) {
+        reset();
+        this.op = newOp;
+        switch (newOp) {
+            case SEARCH:
+                setMemBTreeAccessor();
+                break;
+            case DISKORDERSCAN:
+            case UPDATE:
+                // Attention: It is important to leave the leafFrame and
+                // leafFrameFactory of the memBTree as is when doing an update.
+                // Update will only be set if a previous attempt to delete or
+                // insert failed, so we must preserve the semantics of the
+                // previously requested operation.
+                setMemBTreeAccessor();
+                return;
+            case UPSERT:
+            case INSERT:
+                setInsertMode();
+                break;
+            case PHYSICALDELETE:
+            case DELETE:
+                setDeleteMode();
+                break;
+        }
+    }
+
+    private void setMemBTreeAccessor() {
+        if (memBTreeAccessor == null) {
+            memBTreeAccessor = (BTree.BTreeAccessor) memBTree.createAccessor(modificationCallback,
+                    NoOpOperationCallback.INSTANCE);
+            memBTreeOpCtx = memBTreeAccessor.getOpContext();
+        }
+    }
+
+    public void setInsertMode() {
+        setMemBTreeAccessor();
+        memBTreeOpCtx.leafFrame = insertLeafFrame;
+        memBTreeOpCtx.leafFrameFactory = insertLeafFrameFactory;
+    }
+
+    public void setDeleteMode() {
+        setMemBTreeAccessor();
+        memBTreeOpCtx.leafFrame = deleteLeafFrame;
+        memBTreeOpCtx.leafFrameFactory = deleteLeafFrameFactory;
+    }
+
+    @Override
+    public void reset() {
+        componentHolder.clear();
+    }
+
+    public IndexOperation getOperation() {
+        return op;
+    }
+
+    @Override
+    public List<ILSMComponent> getComponentHolder() {
+        return componentHolder;
+    }
+
+    @Override
+    public ISearchOperationCallback getSearchOperationCallback() {
+        return searchCallback;
+    }
+
+    @Override
+    public IModificationOperationCallback getModificationCallback() {
+        return modificationCallback;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
new file mode 100644
index 0000000..0204833
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -0,0 +1,224 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.ListIterator;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public class LSMBTreePointSearchCursor implements ITreeIndexCursor {
+
+    private IIndexCursor[] rangeCursors;
+    private final ILSMIndexOperationContext opCtx;
+    private ISearchOperationCallback searchCallback;
+    private RangePredicate predicate;
+    private IIndexAccessor memBTreeAccessor;
+    private boolean includeMemComponent;
+    private int numBTrees;
+    private IIndexAccessor[] bTreeAccessors;
+    private ILSMHarness lsmHarness;
+    private boolean nextHasBeenCalled;
+    private boolean foundTuple;
+    private ITupleReference frameTuple;
+
+    public LSMBTreePointSearchCursor(ILSMIndexOperationContext opCtx) {
+        this.opCtx = opCtx;
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        if (nextHasBeenCalled) {
+            return false;
+        } else if (foundTuple) {
+            return true;
+        }
+        boolean reconciled = false;
+        for (int i = 0; i < numBTrees; ++i) {
+            bTreeAccessors[i].search(rangeCursors[i], predicate);
+            if (rangeCursors[i].hasNext()) {
+                rangeCursors[i].next();
+                // We use the predicate's to lock the key instead of the tuple that we get from cursor to avoid copying the tuple when we do the "unlatch dance"
+                if (reconciled || searchCallback.proceed(predicate.getLowKey())) {
+                    // if proceed is successful, then there's no need for doing the "unlatch dance"
+                    if (((ILSMTreeTupleReference) rangeCursors[i].getTuple()).isAntimatter()) {
+                        searchCallback.cancel(predicate.getLowKey());
+                        rangeCursors[i].close();
+                        return false;
+                    } else {
+                        frameTuple = rangeCursors[i].getTuple();
+                        foundTuple = true;
+                        return true;
+                    }
+                }
+                if (i == 0 && includeMemComponent) {
+                    // unlatch/unpin
+                    rangeCursors[i].reset();
+                    searchCallback.reconcile(predicate.getLowKey());
+                    reconciled = true;
+
+                    // retraverse
+                    memBTreeAccessor.search(rangeCursors[i], predicate);
+                    if (rangeCursors[i].hasNext()) {
+                        rangeCursors[i].next();
+                        if (((ILSMTreeTupleReference) rangeCursors[i].getTuple()).isAntimatter()) {
+                            searchCallback.cancel(predicate.getLowKey());
+                            rangeCursors[i].close();
+                            return false;
+                        } else {
+                            frameTuple = rangeCursors[i].getTuple();
+                            foundTuple = true;
+                            return true;
+                        }
+                    } else {
+                        rangeCursors[i].close();
+                    }
+                } else {
+                    frameTuple = rangeCursors[i].getTuple();
+                    searchCallback.reconcile(frameTuple);
+                    foundTuple = true;
+                    return true;
+                }
+            } else {
+                rangeCursors[i].close();
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public void reset() throws HyracksDataException, IndexException {
+        if (rangeCursors != null) {
+            for (int i = 0; i < rangeCursors.length; ++i) {
+                rangeCursors[i].reset();
+            }
+        }
+        rangeCursors = null;
+        nextHasBeenCalled = false;
+        foundTuple = false;
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
+        includeMemComponent = lsmInitialState.getIncludeMemComponent();
+        lsmHarness = lsmInitialState.getLSMHarness();
+        searchCallback = lsmInitialState.getSearchOperationCallback();
+        memBTreeAccessor = lsmInitialState.getMemBTreeAccessor();
+        predicate = (RangePredicate) lsmInitialState.getSearchPredicate();
+
+        numBTrees = lsmInitialState.getNumBTrees();
+        rangeCursors = new IIndexCursor[numBTrees];
+        int i = 0;
+        if (includeMemComponent) {
+            // No need for a bloom filter for the in-memory BTree.
+            IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getLeafFrameFactory().createFrame();
+            rangeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
+            ++i;
+        }
+        for (; i < numBTrees; ++i) {
+            IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getLeafFrameFactory().createFrame();
+            rangeCursors[i] = new BloomFilterAwareBTreePointSearchCursor(leafFrame, false,
+                    ((LSMBTreeImmutableComponent) lsmInitialState.getOperationalComponents().get(i)).getBloomFilter());
+        }
+
+        bTreeAccessors = new IIndexAccessor[numBTrees];
+        int cursorIx = 0;
+        ListIterator<ILSMComponent> btreesIter = lsmInitialState.getOperationalComponents().listIterator();
+        if (includeMemComponent) {
+            bTreeAccessors[cursorIx] = memBTreeAccessor;
+            ++cursorIx;
+            btreesIter.next();
+        }
+
+        while (btreesIter.hasNext()) {
+            BTree diskBTree = ((LSMBTreeImmutableComponent) btreesIter.next()).getBTree();
+            bTreeAccessors[cursorIx] = diskBTree.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            cursorIx++;
+        }
+        nextHasBeenCalled = false;
+        foundTuple = false;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        nextHasBeenCalled = true;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (lsmHarness != null) {
+            try {
+                for (int i = 0; i < rangeCursors.length; i++) {
+                    rangeCursors[i].close();
+                }
+                rangeCursors = null;
+            } finally {
+                lsmHarness.endSearch(opCtx);
+            }
+        }
+        nextHasBeenCalled = false;
+        foundTuple = false;
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return frameTuple;
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        // do nothing
+        return null;
+    }
+
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        // do nothing
+
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        // do nothing
+
+    }
+
+    @Override
+    public boolean exclusiveLatchNodes() {
+        return false;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
new file mode 100644
index 0000000..4255dbf
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -0,0 +1,214 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import java.util.Iterator;
+import java.util.ListIterator;
+
+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.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
+
+public class LSMBTreeRangeSearchCursor extends LSMIndexSearchCursor {
+    private final ArrayTupleReference copyTuple;
+    private final RangePredicate reusablePred;
+
+    private ISearchOperationCallback searchCallback;
+    private RangePredicate predicate;
+    private IIndexAccessor memBTreeAccessor;
+    private ArrayTupleBuilder tupleBuilder;
+    private boolean proceed = true;
+
+    public LSMBTreeRangeSearchCursor(ILSMIndexOperationContext opCtx) {
+        super(opCtx);
+        this.copyTuple = new ArrayTupleReference();
+        this.reusablePred = new RangePredicate(null, null, true, true, null, null);
+    }
+
+    @Override
+    public void reset() throws HyracksDataException, IndexException {
+        super.reset();
+        proceed = true;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        outputElement = outputPriorityQueue.poll();
+        needPush = true;
+        proceed = false;
+    }
+
+    protected void checkPriorityQueue() throws HyracksDataException, IndexException {
+        while (!outputPriorityQueue.isEmpty() || needPush == true) {
+            if (!outputPriorityQueue.isEmpty()) {
+                PriorityQueueElement checkElement = outputPriorityQueue.peek();
+                if (proceed && !searchCallback.proceed(checkElement.getTuple())) {
+                    if (includeMemComponent) {
+                        PriorityQueueElement inMemElement = null;
+                        boolean inMemElementFound = false;
+                        // scan the PQ for the in-memory component's element
+                        Iterator<PriorityQueueElement> it = outputPriorityQueue.iterator();
+                        while (it.hasNext()) {
+                            inMemElement = it.next();
+                            if (inMemElement.getCursorIndex() == 0) {
+                                inMemElementFound = true;
+                                it.remove();
+                                break;
+                            }
+                        }
+                        if (inMemElementFound) {
+                            // copy the in-mem tuple
+                            if (tupleBuilder == null) {
+                                tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+                            }
+                            TupleUtils.copyTuple(tupleBuilder, inMemElement.getTuple(), cmp.getKeyFieldCount());
+                            copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+
+                            // unlatch/unpin
+                            rangeCursors[0].reset();
+
+                            // reconcile
+                            if (checkElement.getCursorIndex() == 0) {
+                                searchCallback.reconcile(copyTuple);
+                            } else {
+                                searchCallback.reconcile(checkElement.getTuple());
+                            }
+                            // retraverse
+                            reusablePred.setLowKey(copyTuple, true);
+                            memBTreeAccessor.search(rangeCursors[0], reusablePred);
+                            pushIntoPriorityQueue(inMemElement);
+                            if (cmp.compare(copyTuple, inMemElement.getTuple()) != 0) {
+                                searchCallback.cancel(copyTuple);
+                                continue;
+                            }
+                        } else {
+                            // the in-memory cursor is exhausted
+                            searchCallback.reconcile(checkElement.getTuple());
+                        }
+                    } else {
+                        searchCallback.reconcile(checkElement.getTuple());
+                    }
+                }
+                // If there is no previous tuple or the previous tuple can be ignored
+                if (outputElement == null) {
+                    if (isDeleted(checkElement)) {
+                        // If the key has been deleted then pop it and set needPush to true.
+                        // We cannot push immediately because the tuple may be
+                        // modified if hasNext() is called
+                        outputElement = outputPriorityQueue.poll();
+                        searchCallback.cancel(checkElement.getTuple());
+                        needPush = true;
+                        proceed = false;
+                    } else {
+                        break;
+                    }
+                } else {
+                    // Compare the previous tuple and the head tuple in the PQ
+                    if (compare(cmp, outputElement.getTuple(), checkElement.getTuple()) == 0) {
+                        // If the previous tuple and the head tuple are
+                        // identical
+                        // then pop the head tuple and push the next tuple from
+                        // the tree of head tuple
+
+                        // the head element of PQ is useless now
+                        PriorityQueueElement e = outputPriorityQueue.poll();
+                        pushIntoPriorityQueue(e);
+                    } else {
+                        // If the previous tuple and the head tuple are different
+                        // the info of previous tuple is useless
+                        if (needPush == true) {
+                            pushIntoPriorityQueue(outputElement);
+                            needPush = false;
+                        }
+                        proceed = true;
+                        outputElement = null;
+                    }
+                }
+            } else {
+                // the priority queue is empty and needPush
+                pushIntoPriorityQueue(outputElement);
+                needPush = false;
+                outputElement = null;
+                proceed = true;
+            }
+        }
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException,
+            IndexException {
+        LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
+        cmp = lsmInitialState.getOriginalKeyComparator();
+        includeMemComponent = lsmInitialState.getIncludeMemComponent();
+        operationalComponents = lsmInitialState.getOperationalComponents();
+        lsmHarness = lsmInitialState.getLSMHarness();
+        searchCallback = lsmInitialState.getSearchOperationCallback();
+        memBTreeAccessor = lsmInitialState.getMemBTreeAccessor();
+        predicate = (RangePredicate) lsmInitialState.getSearchPredicate();
+        reusablePred.setLowKeyComparator(cmp);
+        reusablePred.setHighKey(predicate.getHighKey(), predicate.isHighKeyInclusive());
+        reusablePred.setHighKeyComparator(predicate.getHighKeyComparator());
+
+        int numBTrees = lsmInitialState.getNumBTrees();
+        rangeCursors = new IIndexCursor[numBTrees];
+        for (int i = 0; i < numBTrees; i++) {
+            IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) lsmInitialState.getLeafFrameFactory().createFrame();
+            rangeCursors[i] = new BTreeRangeSearchCursor(leafFrame, false);
+        }
+        setPriorityQueueComparator();
+
+        int cursorIx = 0;
+        ListIterator<ILSMComponent> btreesIter = operationalComponents.listIterator();
+        if (includeMemComponent) {
+            // Open cursor of in-memory BTree at index 0.
+            memBTreeAccessor.search(rangeCursors[cursorIx], searchPred);
+            // Skip 0 because it is the in-memory BTree.
+            ++cursorIx;
+            btreesIter.next();
+        }
+
+        // Open cursors of on-disk BTrees.
+        int numDiskComponents = includeMemComponent ? numBTrees - 1 : numBTrees;
+        ITreeIndexAccessor[] diskBTreeAccessors = new ITreeIndexAccessor[numDiskComponents];
+        int diskBTreeIx = 0;
+        while (btreesIter.hasNext()) {
+            BTree diskBTree = (BTree) ((LSMBTreeImmutableComponent) btreesIter.next()).getBTree();
+            diskBTreeAccessors[diskBTreeIx] = diskBTree.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            diskBTreeAccessors[diskBTreeIx].search(rangeCursors[cursorIx], searchPred);
+            cursorIx++;
+            diskBTreeIx++;
+        }
+        initPriorityQueue();
+        proceed = true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
new file mode 100644
index 0000000..6872520
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public class LSMBTreeSearchCursor implements ITreeIndexCursor {
+
+    public enum LSMBTreeSearchType {
+        POINT,
+        RANGE
+    }
+
+    private final LSMBTreePointSearchCursor pointCursor;
+    private final LSMBTreeRangeSearchCursor rangeCursor;
+    private ITreeIndexCursor currentCursor;
+
+    public LSMBTreeSearchCursor(ILSMIndexOperationContext opCtx) {
+        pointCursor = new LSMBTreePointSearchCursor(opCtx);
+        rangeCursor = new LSMBTreeRangeSearchCursor(opCtx);
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws IndexException,
+            HyracksDataException {
+
+        LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
+
+        LSMBTreeSearchType searchType = LSMBTreeSearchType.RANGE;
+        RangePredicate btreePred = (RangePredicate) searchPred;
+        if (btreePred.getLowKey() != null && btreePred.getHighKey() != null) {
+            if (btreePred.isLowKeyInclusive() && btreePred.isHighKeyInclusive()) {
+                if (btreePred.getLowKeyComparator().getKeyFieldCount() == btreePred.getHighKeyComparator()
+                        .getKeyFieldCount()) {
+                    if (btreePred.getLowKeyComparator().getKeyFieldCount() == lsmInitialState
+                            .getOriginalKeyComparator().getKeyFieldCount()) {
+                        if (lsmInitialState.getOriginalKeyComparator().compare(btreePred.getLowKey(),
+                                btreePred.getHighKey()) == 0) {
+                            searchType = LSMBTreeSearchType.POINT;
+                        }
+                    }
+                }
+            }
+        }
+        switch (searchType) {
+            case POINT:
+                currentCursor = pointCursor;
+                break;
+            case RANGE:
+                currentCursor = rangeCursor;
+                break;
+            default:
+                throw new HyracksDataException("Wrong search type");
+        }
+        currentCursor.open(lsmInitialState, searchPred);
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        return currentCursor.hasNext();
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        currentCursor.next();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (currentCursor != null) {
+            currentCursor.close();
+        }
+        currentCursor = null;
+    }
+
+    @Override
+    public void reset() throws HyracksDataException, IndexException {
+        if (currentCursor != null) {
+            currentCursor.reset();
+        }
+        currentCursor = null;
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return currentCursor.getTuple();
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        return currentCursor.getPage();
+    }
+
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        currentCursor.setBufferCache(bufferCache);
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        currentCursor.setFileId(fileId);
+
+    }
+
+    @Override
+    public boolean exclusiveLatchNodes() {
+        return currentCursor.exclusiveLatchNodes();
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriter.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriter.java
new file mode 100644
index 0000000..2a66644
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriter.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class LSMBTreeCopyTupleWriter extends LSMBTreeTupleWriter {
+	public LSMBTreeCopyTupleWriter(ITypeTraits[] typeTraits, int numKeyFields){
+		// Third parameter is never used locally, just give false.
+	    super(typeTraits, numKeyFields, false);
+	}
+	
+	@Override
+    public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
+		int tupleSize = bytesRequired(tuple);
+		byte[] buf = tuple.getFieldData(0);
+		int tupleStartOff = ((LSMBTreeTupleReference)tuple).getTupleStart();
+		System.arraycopy(buf, tupleStartOff, targetBuf, targetOff, tupleSize);
+        return tupleSize;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriterFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriterFactory.java
new file mode 100644
index 0000000..b73e9af
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeCopyTupleWriterFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+
+public class LSMBTreeCopyTupleWriterFactory extends TypeAwareTupleWriterFactory {
+	private static final long serialVersionUID = 1L;
+	private final ITypeTraits[] typeTraits;
+	private final int numKeyFields;
+	
+	public LSMBTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits, int numKeyFields) {
+		super(typeTraits);
+		this.typeTraits = typeTraits;
+		this.numKeyFields = numKeyFields;
+	}
+
+	@Override
+	public ITreeIndexTupleWriter createTupleWriter() {
+		return new LSMBTreeCopyTupleWriter(typeTraits, numKeyFields);
+	}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleReference.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleReference.java
new file mode 100644
index 0000000..2c55792
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleReference.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.tuples;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+
+public class LSMBTreeTupleReference extends TypeAwareTupleReference implements ILSMTreeTupleReference {
+
+    // Indicates whether the last call to setFieldCount() was initiated by
+    // by the outside or whether it was called internally to set up an
+    // antimatter tuple.
+    private boolean resetFieldCount = false;
+    private final int numKeyFields;
+    
+    public LSMBTreeTupleReference(ITypeTraits[] typeTraits, int numKeyFields) {
+		super(typeTraits);
+		this.numKeyFields = numKeyFields;
+	}
+
+    public void setFieldCount(int fieldCount) {
+        super.setFieldCount(fieldCount);
+        // Don't change the fieldCount in reset calls.
+        resetFieldCount = false;
+    }
+
+    @Override
+    public void setFieldCount(int fieldStartIndex, int fieldCount) {
+        super.setFieldCount(fieldStartIndex, fieldCount);
+        // Don't change the fieldCount in reset calls.
+        resetFieldCount = false;
+    }
+    
+    @Override
+    public void resetByTupleOffset(ByteBuffer buf, int tupleStartOff) {
+        this.buf = buf;
+        this.tupleStartOff = tupleStartOff;
+        if (numKeyFields != typeTraits.length) {
+            if (isAntimatter()) {
+                setFieldCount(numKeyFields);
+                // Reset the original field count for matter tuples.
+                resetFieldCount = true;
+            } else {
+                if (resetFieldCount) {
+                    setFieldCount(typeTraits.length);
+                }
+            }
+        }
+        super.resetByTupleOffset(buf, tupleStartOff);
+    }
+    
+    @Override
+    public void resetByTupleIndex(ITreeIndexFrame frame, int tupleIndex) {
+        resetByTupleOffset(frame.getBuffer(), frame.getTupleOffset(tupleIndex));
+    }
+    
+	@Override
+	protected int getNullFlagsBytes() {
+		// +1.0 is for matter/antimatter bit.
+		return (int) Math.ceil((fieldCount + 1.0) / 8.0);
+    }
+
+	@Override
+	public boolean isAntimatter() {
+	      // Check if the leftmost bit is 0 or 1.
+		final byte mask = (byte) (1 << 7);
+		if ((buf.array()[tupleStartOff] & mask) != 0) {
+		    return true;
+		}
+		return false;
+	}
+	
+    public int getTupleStart() {
+    	return tupleStartOff;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java
new file mode 100644
index 0000000..12aca6f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriter.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
+
+public class LSMBTreeTupleWriter extends TypeAwareTupleWriter {
+    private final boolean isAntimatter;
+    private final int numKeyFields;
+
+    public LSMBTreeTupleWriter(ITypeTraits[] typeTraits, int numKeyFields, boolean isAntimatter) {
+        super(typeTraits);
+        this.numKeyFields = numKeyFields;
+        this.isAntimatter = isAntimatter;
+    }
+
+    @Override
+    public int bytesRequired(ITupleReference tuple) {
+        if (isAntimatter) {
+            // Only requires space for the key fields.
+            return super.bytesRequired(tuple, 0, numKeyFields);
+        } else {
+            return super.bytesRequired(tuple);
+        }
+    }
+
+    @Override
+    public int getCopySpaceRequired(ITupleReference tuple) {
+        return super.bytesRequired(tuple);
+    }
+
+    @Override
+    public ITreeIndexTupleReference createTupleReference() {
+        return new LSMBTreeTupleReference(typeTraits, numKeyFields);
+    }
+
+    @Override
+    protected int getNullFlagsBytes(int numFields) {
+        // +1.0 is for matter/antimatter bit.
+        return (int) Math.ceil(((double) numFields + 1.0) / 8.0);
+    }
+
+    @Override
+    protected int getNullFlagsBytes(ITupleReference tuple) {
+        // +1.0 is for matter/antimatter bit.
+        return (int) Math.ceil(((double) tuple.getFieldCount() + 1.0) / 8.0);
+    }
+
+    @Override
+    public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
+        int bytesWritten = -1;
+        if (isAntimatter) {
+            bytesWritten = super.writeTupleFields(tuple, 0, numKeyFields, targetBuf, targetOff);
+            setAntimatterBit(targetBuf, targetOff);
+        } else {
+            bytesWritten = super.writeTuple(tuple, targetBuf, targetOff);
+        }
+        return bytesWritten;
+    }
+
+    private void setAntimatterBit(byte[] targetBuf, int targetOff) {
+        // Set leftmost bit to 1.
+        targetBuf[targetOff] = (byte) (targetBuf[targetOff] | (1 << 7));
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java
new file mode 100644
index 0000000..8eb24a0
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTupleWriterFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+
+public class LSMBTreeTupleWriterFactory extends TypeAwareTupleWriterFactory {
+
+	private static final long serialVersionUID = 1L;
+	private final ITypeTraits[] typeTraits;
+	private final int numKeyFields;
+	private final boolean isDelete;
+	
+	public LSMBTreeTupleWriterFactory(ITypeTraits[] typeTraits, int numKeyFields, boolean isDelete) {
+		super(typeTraits);
+		this.typeTraits = typeTraits;
+		this.numKeyFields = numKeyFields;
+		this.isDelete = isDelete;
+	}
+
+	@Override
+	public ITreeIndexTupleWriter createTupleWriter() {
+		return new LSMBTreeTupleWriter(typeTraits, numKeyFields, isDelete);
+	}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
new file mode 100644
index 0000000..ac20b6d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.util;
+
+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;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManagerFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMBTreeUtils {
+
+    public static LSMBTree createLSMTree(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        return createLSMTree(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache, diskFileMapProvider,
+                typeTraits, cmpFactories, bloomFilterKeyFields, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider, 0);
+    }
+
+    public static LSMBTree createLSMTree(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int startIODeviceIndex) {
+        LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+                cmpFactories.length, false);
+        LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
+                cmpFactories.length, true);
+        LSMBTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMBTreeCopyTupleWriterFactory(typeTraits,
+                cmpFactories.length);
+        ITreeIndexFrameFactory insertLeafFrameFactory = new BTreeNSMLeafFrameFactory(insertTupleWriterFactory);
+        ITreeIndexFrameFactory copyTupleLeafFrameFactory = new BTreeNSMLeafFrameFactory(copyTupleWriterFactory);
+        ITreeIndexFrameFactory deleteLeafFrameFactory = new BTreeNSMLeafFrameFactory(deleteTupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(insertTupleWriterFactory);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        IFreePageManagerFactory freePageManagerFactory = new LinkedListFreePageManagerFactory(diskBufferCache,
+                metaFrameFactory);
+
+        TreeIndexFactory<BTree> diskBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, interiorFrameFactory, copyTupleLeafFrameFactory, cmpFactories,
+                typeTraits.length);
+        TreeIndexFactory<BTree> bulkLoadBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
+
+        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
+                bloomFilterKeyFields);
+
+        ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, diskFileMapProvider, file,
+                diskBTreeFactory, startIODeviceIndex);
+
+        LSMBTree lsmTree = new LSMBTree(memBufferCache, memFreePageManager, interiorFrameFactory,
+                insertLeafFrameFactory, deleteLeafFrameFactory, fileNameManager, diskBTreeFactory,
+                bulkLoadBTreeFactory, bloomFilterFactory, diskFileMapProvider, typeTraits.length, cmpFactories,
+                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        return lsmTree;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/pom.xml b/hyracks/hyracks-storage-am-lsm-common/pom.xml
new file mode 100644
index 0000000..ee62131
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/pom.xml
@@ -0,0 +1,47 @@
+<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>
+  <artifactId>hyracks-storage-am-lsm-common</artifactId>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-common</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-bloomfilter</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-btree</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>  	
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IInMemoryBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IInMemoryBufferCache.java
new file mode 100644
index 0000000..082ad2f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/IInMemoryBufferCache.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCacheInternal;
+
+public interface IInMemoryBufferCache extends IBufferCacheInternal {
+    public void open();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponent.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponent.java
new file mode 100644
index 0000000..fa00f85
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponent.java
@@ -0,0 +1,10 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+
+public interface ILSMComponent {
+    public boolean threadEnter(LSMOperationType opType) throws InterruptedException;
+
+    public void threadExit(LSMOperationType opType, boolean failedOperation) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponentFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponentFactory.java
new file mode 100644
index 0000000..1f3a2b7
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMComponentFactory.java
@@ -0,0 +1,13 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public interface ILSMComponentFactory {
+    public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) throws IndexException,
+            HyracksDataException;
+
+    public IBufferCache getBufferCache();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMHarness.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMHarness.java
new file mode 100644
index 0000000..04d7fa3
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMHarness.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+public interface ILSMHarness {
+
+    public void forceModify(ILSMIndexOperationContext ctx, ITupleReference tuple) throws HyracksDataException,
+            IndexException;
+
+    public boolean modify(ILSMIndexOperationContext ctx, boolean tryOperation, ITupleReference tuple)
+            throws HyracksDataException, IndexException;
+
+    public void noOp(ILSMIndexOperationContext ctx) throws HyracksDataException;
+
+    public void search(ILSMIndexOperationContext ctx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException;
+
+    public void endSearch(ILSMIndexOperationContext ctx) throws HyracksDataException;
+
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException;
+
+    public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException,
+            IndexException;
+
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException;
+
+    public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException,
+            IndexException;
+
+    public void addBulkLoadedComponent(ILSMComponent index) throws HyracksDataException, IndexException;
+
+    public ILSMOperationTracker getOperationTracker();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
new file mode 100644
index 0000000..15fdc6e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
@@ -0,0 +1,17 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+public interface ILSMIOOperation {
+    public Set<IODeviceHandle> getReadDevices();
+
+    public Set<IODeviceHandle> getWriteDevices();
+
+    public void perform() throws HyracksDataException, IndexException;
+
+    public ILSMIOOperationCallback getCallback();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
new file mode 100644
index 0000000..bf12583
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
@@ -0,0 +1,14 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ILSMIOOperationCallback {
+    public void beforeOperation() throws HyracksDataException;
+
+    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+            throws HyracksDataException;
+
+    public void afterFinalize(ILSMComponent newComponent) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
new file mode 100644
index 0000000..52361ee
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.io.Serializable;
+
+public interface ILSMIOOperationCallbackFactory extends Serializable {
+    public ILSMIOOperationCallback createIOOperationCallback(Object syncObj);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackProvider.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackProvider.java
new file mode 100644
index 0000000..b5c9741
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackProvider.java
@@ -0,0 +1,5 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+public interface ILSMIOOperationCallbackProvider {
+    public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationScheduler.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationScheduler.java
new file mode 100644
index 0000000..6d96562
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationScheduler.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ILSMIOOperationScheduler {
+    public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java
new file mode 100644
index 0000000..9737728
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerProvider.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public interface ILSMIOOperationSchedulerProvider extends Serializable {
+    public ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
new file mode 100644
index 0000000..cff47bb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMHarness;
+
+/**
+ * Methods to be implemented by an LSM index, which are called from {@link LSMHarness}.
+ * The implementations of the methods below should be thread agnostic.
+ * Synchronization of LSM operations like updates/searches/flushes/merges are
+ * done by the {@link LSMHarness}. For example, a flush() implementation should only
+ * create and return the new on-disk component, ignoring the fact that
+ * concurrent searches/updates/merges may be ongoing.
+ */
+public interface ILSMIndex extends IIndex {
+    
+    public void deactivate(boolean flushOnExit) throws HyracksDataException;
+
+    public ILSMIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback);
+
+    public boolean getFlushStatus(ILSMIndex index);
+
+    public ILSMOperationTracker getOperationTracker();
+
+    public ILSMIOOperationScheduler getIOScheduler();
+
+    public List<ILSMComponent> getImmutableComponents();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
new file mode 100644
index 0000000..a85c24b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+/**
+ * Client handle for performing operations
+ * (insert/delete/update/search/diskorderscan/merge/flush) on an {@link ILSMHarness}.
+ * An {@link ILSMIndexAccessor} is not thread safe, but different {@link ILSMIndexAccessor}s
+ * can concurrently operate on the same {@link ILSMIndex} (i.e., the {@link ILSMIndex} must allow
+ * concurrent operations).
+ */
+public interface ILSMIndexAccessor extends IIndexAccessor {
+    public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException;
+
+    public void scheduleMerge(ILSMIOOperationCallback callback) throws HyracksDataException, IndexException;
+
+    /**
+     * Deletes the tuple from the memory component only.
+     * 
+     * @throws HyracksDataException
+     * @throws IndexException
+     */
+    public void physicalDelete(ITupleReference tuple) throws HyracksDataException, IndexException;
+
+    /**
+     * Attempts to insert the given tuple.
+     * If the insert would have to wait for a flush to complete, then this method returns false to
+     * allow the caller to avoid potential deadlock situations.
+     * Otherwise, returns true (insert was successful).
+     * 
+     * @param tuple
+     *            Tuple to be inserted.
+     * @throws HyracksDataException
+     *             If the BufferCache throws while un/pinning or un/latching.
+     * @throws IndexException
+     *             If an index-specific constraint is violated, e.g., the key
+     *             already exists.
+     */
+    public boolean tryInsert(ITupleReference tuple) throws HyracksDataException, IndexException;
+
+    /**
+     * Attempts to delete the given tuple.
+     * If the delete would have to wait for a flush to complete, then this method returns false to
+     * allow the caller to avoid potential deadlock situations.
+     * Otherwise, returns true (delete was successful).
+     * 
+     * @param tuple
+     *            Tuple to be deleted.
+     * @throws HyracksDataException
+     *             If the BufferCache throws while un/pinning or un/latching.
+     * @throws IndexException
+     *             If there is no matching tuple in the index.
+     */
+    public boolean tryDelete(ITupleReference tuple) throws HyracksDataException, IndexException;
+
+    /**
+     * Attempts to update the given tuple.
+     * If the update would have to wait for a flush to complete, then this method returns false to
+     * allow the caller to avoid potential deadlock situations.
+     * Otherwise, returns true (update was successful).
+     * 
+     * @param tuple
+     *            Tuple whose match in the index is to be update with the given
+     *            tuples contents.
+     * @throws HyracksDataException
+     *             If the BufferCache throws while un/pinning or un/latching.
+     * @throws IndexException
+     *             If there is no matching tuple in the index.
+     */
+    public boolean tryUpdate(ITupleReference tuple) throws HyracksDataException, IndexException;
+
+    /**
+     * This operation is only supported by indexes with the notion of a unique key.
+     * If tuple's key already exists, then this operation attempts to performs an update.
+     * Otherwise, it attempts to perform an insert.
+     * If the operation would have to wait for a flush to complete, then this method returns false to
+     * allow the caller to avoid potential deadlock situations.
+     * Otherwise, returns true (insert/update was successful).
+     * 
+     * @param tuple
+     *            Tuple to be deleted.
+     * @throws HyracksDataException
+     *             If the BufferCache throws while un/pinning or un/latching.
+     * @throws IndexException
+     *             If there is no matching tuple in the index.
+     */
+    public boolean tryUpsert(ITupleReference tuple) throws HyracksDataException, IndexException;
+
+    public void forcePhysicalDelete(ITupleReference tuple) throws HyracksDataException, IndexException;
+
+    public void forceInsert(ITupleReference tuple) throws HyracksDataException, IndexException;
+
+    public void forceDelete(ITupleReference tuple) throws HyracksDataException, IndexException;
+
+    /**
+     * This method can be used to increase the number of 'active' operations of an index artificially,
+     * without actually modifying the index.
+     * This method does not block and is guaranteed to trigger the {@link ILSMOperationTracker}'s beforeOperation
+     * and afterOperation calls.
+     * 
+     * @throws HyracksDataException
+     */
+    public void noOp() throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessorInternal.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessorInternal.java
new file mode 100644
index 0000000..8c28d47
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexAccessorInternal.java
@@ -0,0 +1,24 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+
+public interface ILSMIndexAccessorInternal extends ILSMIndexAccessor {
+
+    /**
+     * Force a flush of the in-memory component.
+     * 
+     * @throws HyracksDataException
+     * @throws TreeIndexException
+     */
+    public void flush(ILSMIOOperation operation) throws HyracksDataException, IndexException;
+
+    /**
+     * Merge all on-disk components.
+     * 
+     * @throws HyracksDataException
+     * @throws TreeIndexException
+     */
+    public void merge(ILSMIOOperation operation) throws HyracksDataException, IndexException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java
new file mode 100644
index 0000000..bc922fe
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexFileManager.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.util.Comparator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+
+/**
+ * Provides file names for LSM on-disk components. Also cleans up invalid files.
+ * There are separate methods to get file names for merge and flush because we
+ * need to guarantee the correct order of on-disk components (i.e., the
+ * components produced by flush are always newer than those produced by a
+ * merge).
+ */
+public interface ILSMIndexFileManager {
+    public void createDirs();
+
+    public void deleteDirs();
+
+    public LSMComponentFileReferences getRelFlushFileReference();
+
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
+            throws HyracksDataException;
+
+    public String getBaseDir();
+
+    // Deletes invalid files, and returns list of valid files from baseDir.
+    // The returned valid files are correctly sorted (based on the recency of data). 
+    public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException, IndexException;
+
+    public Comparator<String> getFileNameComparator();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java
new file mode 100644
index 0000000..e98165b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexInternal.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+public interface ILSMIndexInternal extends ILSMIndex {
+    public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback);
+
+    public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException;
+
+    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException;
+
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException;
+
+    public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException;
+
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException;
+
+    public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
+            throws HyracksDataException, IndexException;
+
+    public void addComponent(ILSMComponent index);
+
+    public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents);
+
+    /**
+     * Populates the context's component holder with a snapshot of the components involved in the operation.
+     * 
+     * @param ctx
+     *            - the operation's context
+     */
+    public void getOperationalComponents(ILSMIndexOperationContext ctx);
+
+    public IInMemoryFreePageManager getInMemoryFreePageManager();
+
+    public List<ILSMComponent> getImmutableComponents();
+
+    public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException;
+
+    public void setFlushStatus(ILSMIndex index, boolean needsFlush);
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
new file mode 100644
index 0000000..864d0e7
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+
+public interface ILSMIndexOperationContext extends IIndexOperationContext {
+    public List<ILSMComponent> getComponentHolder();
+
+    public ISearchOperationCallback getSearchOperationCallback();
+
+    public IModificationOperationCallback getModificationCallback();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicy.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicy.java
new file mode 100644
index 0000000..877c6ffd
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicy.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+public interface ILSMMergePolicy {
+    public void diskComponentAdded(ILSMIndex index, int totalNumDiskComponents) throws HyracksDataException, IndexException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicyProvider.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicyProvider.java
new file mode 100644
index 0000000..57a9609
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMMergePolicyProvider.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public interface ILSMMergePolicyProvider extends Serializable {
+    public ILSMMergePolicy getMergePolicy(IHyracksTaskContext ctx);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTracker.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTracker.java
new file mode 100644
index 0000000..c3f1f3e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTracker.java
@@ -0,0 +1,44 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+
+/**
+ * This interface exposes methods for tracking and setting the status of operations for the purpose
+ * of coordinating flushes/merges in {@link ILSMIndex}.
+ * Note that 'operation' below refers to {@link IIndexAccessor} methods.
+ * 
+ * @author zheilbron
+ */
+public interface ILSMOperationTracker {
+
+    /**
+     * An {@link ILSMIndex} will call this method before an operation enters it,
+     * i.e., before any latches are taken.
+     * If tryOperation is true, and the operation would have to wait for a flush,
+     * then this method does not block and returns false.
+     * Otherwise, this method returns true, and the operation is considered 'active' in the index.
+     */
+    public void beforeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException;
+
+    /**
+     * An {@link ILSMIndex} will call this method after an operation has left the index,
+     * i.e., after all relevant latches have been released.
+     * After this method has been called, the operation is still considered 'active',
+     * until the issuer of the operation declares it completed by calling completeOperation().
+     */
+    public void afterOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException;
+
+    /**
+     * This method must be called by whoever is requesting the index operation through an {@link IIndexAccessor}.
+     * The use of this method indicates that the operation is no longer 'active'
+     * for the purpose of coordinating flushes/merges.
+     */
+    public void completeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
new file mode 100644
index 0000000..db7ff6c
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMOperationTrackerFactory.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import java.io.Serializable;
+
+public interface ILSMOperationTrackerFactory extends Serializable {
+    public ILSMOperationTracker createOperationTracker(ILSMIndex index);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMTreeTupleReference.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMTreeTupleReference.java
new file mode 100644
index 0000000..8d82673
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMTreeTupleReference.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.api;
+
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+
+public interface ILSMTreeTupleReference extends ITreeIndexTupleReference {
+    public boolean isAntimatter();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
new file mode 100644
index 0000000..ea7c3b4
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+
+public abstract class AbstractLSMIndexDataflowHelper extends IndexDataflowHelper {
+
+    protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
+    protected static int DEFAULT_MEM_NUM_PAGES = 1000;
+
+    protected final int memPageSize;
+    protected final int memNumPages;
+
+    protected final ILSMMergePolicy mergePolicy;
+    protected final ILSMIOOperationScheduler ioScheduler;
+    protected final ILSMOperationTrackerFactory opTrackerFactory;
+    protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+
+    public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, mergePolicy, opTrackerFactory,
+                ioScheduler, ioOpCallbackProvider);
+    }
+
+    public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(opDesc, ctx, partition);
+        this.memPageSize = memPageSize;
+        this.memNumPages = memNumPages;
+        this.mergePolicy = mergePolicy;
+        this.opTrackerFactory = opTrackerFactory;
+        this.ioScheduler = ioScheduler;
+        this.ioOpCallbackProvider = ioOpCallbackProvider;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
new file mode 100644
index 0000000..a2f2a11
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.dataflow;
+
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+
+public abstract class AbstractLSMIndexDataflowHelperFactory implements IIndexDataflowHelperFactory {
+    protected static final long serialVersionUID = 1L;
+
+    protected final ILSMMergePolicyProvider mergePolicyProvider;
+    protected final ILSMOperationTrackerFactory opTrackerFactory;
+    protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
+    protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    protected final int memPageSize;
+    protected final int memNumPages;
+
+    public AbstractLSMIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+        this.mergePolicyProvider = mergePolicyProvider;
+        this.opTrackerFactory = opTrackerFactory;
+        this.ioSchedulerProvider = ioSchedulerProvider;
+        this.ioOpCallbackProvider = ioOpCallbackProvider;
+        this.memPageSize = memPageSize;
+        this.memNumPages = memNumPages;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
new file mode 100644
index 0000000..baa9648
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.storage.am.lsm.common.dataflow;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexInsertUpdateDeleteOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+
+public class LSMIndexInsertUpdateDeleteOperatorNodePushable extends IndexInsertUpdateDeleteOperatorNodePushable {
+
+    protected FrameTupleAppender appender;
+
+    public LSMIndexInsertUpdateDeleteOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider, IndexOperation op) {
+        super(opDesc, ctx, partition, fieldPermutation, recordDescProvider, op);
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
+        int lastFlushedTupleIndex = 0;
+        int tupleCount = accessor.getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            try {
+                if (tupleFilter != null) {
+                    frameTuple.reset(accessor, i);
+                    if (!tupleFilter.accept(frameTuple)) {
+                        lsmAccessor.noOp();
+                        continue;
+                    }
+                }
+                tuple.reset(accessor, i);
+
+                switch (op) {
+                    case INSERT: {
+                        if (!lsmAccessor.tryInsert(tuple)) {
+                            flushPartialFrame(lastFlushedTupleIndex, i);
+                            lastFlushedTupleIndex = (i == 0) ? 0 : i - 1;
+                            lsmAccessor.insert(tuple);
+                        }
+                        break;
+                    }
+                    case DELETE: {
+                        if (!lsmAccessor.tryDelete(tuple)) {
+                            flushPartialFrame(lastFlushedTupleIndex, i);
+                            lastFlushedTupleIndex = (i == 0) ? 0 : i - 1;
+                            lsmAccessor.delete(tuple);
+                        }
+                        break;
+                    }
+                    case UPSERT: {
+                        if (!lsmAccessor.tryUpsert(tuple)) {
+                            flushPartialFrame(lastFlushedTupleIndex, i);
+                            lastFlushedTupleIndex = (i == 0) ? 0 : i - 1;
+                            lsmAccessor.upsert(tuple);
+                        }
+                        break;
+                    }
+                    case UPDATE: {
+                        if (!lsmAccessor.tryUpdate(tuple)) {
+                            flushPartialFrame(lastFlushedTupleIndex, i);
+                            lastFlushedTupleIndex = (i == 0) ? 0 : i - 1;
+                            lsmAccessor.update(tuple);
+                        }
+                        break;
+                    }
+                    default: {
+                        throw new HyracksDataException("Unsupported operation " + op
+                                + " in tree index InsertUpdateDelete operator");
+                    }
+                }
+            } catch (HyracksDataException e) {
+                throw e;
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        if (lastFlushedTupleIndex == 0) {
+            // No partial flushing was necessary. Forward entire frame.
+            System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
+            FrameUtils.flushFrame(writeBuffer, writer);
+        } else {
+            // Flush remaining partial frame.
+            flushPartialFrame(lastFlushedTupleIndex, tupleCount);
+        }
+    }
+
+    private void flushPartialFrame(int startTupleIndex, int endTupleIndex) throws HyracksDataException {
+        if (appender == null) {
+            appender = new FrameTupleAppender(ctx.getFrameSize());
+        }
+        appender.reset(writeBuffer, true);
+        for (int i = startTupleIndex; i < endTupleIndex; i++) {
+            if (!appender.append(accessor, i)) {
+                throw new IllegalStateException("Failed to append tuple into frame.");
+            }
+        }
+        FrameUtils.flushFrame(writeBuffer, writer);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
new file mode 100644
index 0000000..6297576
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMTreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+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.job.IOperatorDescriptorRegistry;
+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.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
+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;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
+
+public class LSMTreeIndexInsertUpdateDeleteOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int[] fieldPermutation;
+    private final IndexOperation op;
+
+    public LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] fieldPermutation,
+            IndexOperation op, IIndexDataflowHelperFactory dataflowHelperFactory,
+            ITupleFilterFactory tupleFilterFactory, IModificationOperationCallbackFactory modificationOpCallbackProvider) {
+        super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, tupleFilterFactory, false,
+                NoOpLocalResourceFactoryProvider.INSTANCE, NoOpOperationCallbackFactory.INSTANCE,
+                modificationOpCallbackProvider);
+        this.fieldPermutation = fieldPermutation;
+        this.op = op;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new LSMIndexInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
+                recordDescProvider, op);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryBufferCache.java
new file mode 100644
index 0000000..c24c473
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryBufferCache.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.freepage;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+
+/**
+ * In-memory buffer cache that supports two tree indexes.
+ * We assume that the tree indexes have 2 fixed pages, one at index 0 (metadata page), and one at index 1 (root page).
+ */
+public class DualIndexInMemoryBufferCache extends InMemoryBufferCache {
+
+    public DualIndexInMemoryBufferCache(ICacheMemoryAllocator allocator, int pageSize, int numPages) {
+        super(allocator, pageSize, numPages, new TransientFileMapManager());
+    }
+
+    @Override
+    public ICachedPage pin(long dpid, boolean newPage) {
+        int pageId = BufferedFileHandle.getPageId(dpid);
+        int fileId = BufferedFileHandle.getFileId(dpid);
+        if (pageId < pages.length) {
+            // Common case: Return regular page.
+            if (pageId == 0 || pageId == 1) {
+                return pages[pageId + 2 * fileId];
+            } else {
+                return pages[pageId];
+            }
+        } else {
+            // Rare case: Return overflow page, possibly expanding overflow
+            // array.
+            synchronized (overflowPages) {
+                int numNewPages = pageId - pages.length - overflowPages.size() + 1;
+                if (numNewPages > 0) {
+                    ByteBuffer[] buffers = allocator.allocate(pageSize, numNewPages);
+                    for (int i = 0; i < numNewPages; i++) {
+                        CachedPage overflowPage = new CachedPage(pages.length + overflowPages.size(), buffers[i]);
+                        overflowPages.add(overflowPage);
+                    }
+                }
+                return overflowPages.get(pageId - pages.length);
+            }
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryFreePageManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryFreePageManager.java
new file mode 100644
index 0000000..7a2be7f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/DualIndexInMemoryFreePageManager.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.freepage;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+
+/**
+ * In-memory free page manager that supports two tree indexes.
+ * We assume that the tree indexes have 2 fixed pages, one at index 0 (metadata page), and one at index 1 (root page).
+ */
+public class DualIndexInMemoryFreePageManager extends InMemoryFreePageManager {
+
+    public DualIndexInMemoryFreePageManager(int capacity, ITreeIndexMetaDataFrameFactory metaDataFrameFactory) {
+        super(capacity, metaDataFrameFactory);
+        // We start the currentPageId from 3, because the RTree uses
+        // the first page as metadata page, and the second page as root page.
+        // And the BTree uses the third page as metadata, and the third page as root page 
+        // (when returning free pages we first increment, then get)
+        currentPageId.set(3);
+    }
+
+    @Override
+    public void init(ITreeIndexMetaDataFrame metaFrame, int currentMaxPage) throws HyracksDataException {
+        currentPageId.set(3);
+    }
+
+    public int getCapacity() {
+        return capacity - 4;
+    }
+
+    public void reset() {
+        currentPageId.set(3);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryBufferCache.java
new file mode 100644
index 0000000..66d8ec2
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryBufferCache.java
@@ -0,0 +1,206 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.freepage;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPageInternal;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class InMemoryBufferCache implements IInMemoryBufferCache {
+    protected final ICacheMemoryAllocator allocator;
+    protected final IFileMapManager fileMapManager;
+    protected final int pageSize;
+    protected final int numPages;
+    protected final List<CachedPage> overflowPages = new ArrayList<CachedPage>();
+    protected CachedPage[] pages;
+
+    public InMemoryBufferCache(ICacheMemoryAllocator allocator, int pageSize, int numPages,
+            IFileMapManager fileMapManager) {
+        this.allocator = allocator;
+        this.fileMapManager = fileMapManager;
+        this.pageSize = pageSize;
+        this.numPages = numPages;
+    }
+
+    public void open() {
+        pages = new CachedPage[numPages];
+        ByteBuffer[] buffers = allocator.allocate(pageSize, numPages);
+        for (int i = 0; i < buffers.length; ++i) {
+            pages[i] = new CachedPage(i, buffers[i]);
+        }
+    }
+
+    @Override
+    public ICachedPage pin(long dpid, boolean newPage) {
+        int pageId = BufferedFileHandle.getPageId(dpid);
+        if (pageId < pages.length) {
+            // Common case: Return regular page.
+            return pages[pageId];
+        } else {
+            // Rare case: Return overflow page, possibly expanding overflow array.
+            synchronized (overflowPages) {
+                int numNewPages = pageId - pages.length - overflowPages.size() + 1;
+                if (numNewPages > 0) {
+                    ByteBuffer[] buffers = allocator.allocate(pageSize, numNewPages);
+                    for (int i = 0; i < numNewPages; i++) {
+                        CachedPage overflowPage = new CachedPage(pages.length + overflowPages.size(), buffers[i]);
+                        overflowPages.add(overflowPage);
+                    }
+                }
+                return overflowPages.get(pageId - pages.length);
+            }
+        }
+    }
+
+    @Override
+    public ICachedPage tryPin(long dpid) throws HyracksDataException {
+        return pin(dpid, false);
+    }
+
+    @Override
+    public int getPageSize() {
+        return pageSize;
+    }
+
+    @Override
+    public int getNumPages() {
+        return numPages;
+    }
+
+    @Override
+    public ICachedPageInternal getPage(int cpid) {
+        return pages[cpid];
+    }
+
+    public int getNumOverflowPages() {
+        return overflowPages.size();
+    }
+
+    @Override
+    public void createFile(FileReference fileRef) throws HyracksDataException {
+        synchronized (fileMapManager) {
+            fileMapManager.registerFile(fileRef);
+        }
+    }
+
+    @Override
+    public void openFile(int fileId) throws HyracksDataException {
+        // Do nothing.
+    }
+
+    @Override
+    public void closeFile(int fileId) throws HyracksDataException {
+        // Do nothing.
+    }
+
+    @Override
+    public void deleteFile(int fileId, boolean flushDirtyPages) throws HyracksDataException {
+        synchronized (fileMapManager) {
+            fileMapManager.unregisterFile(fileId);
+        }
+    }
+
+    @Override
+    public void unpin(ICachedPage page) throws HyracksDataException {
+        // Do Nothing.
+    }
+
+    @Override
+    public void close() {
+        for (int i = 0; i < numPages; ++i) {
+            pages[i] = null;
+        }
+        overflowPages.clear();
+    }
+
+    public class CachedPage implements ICachedPageInternal {
+        private final int cpid;
+        private final ByteBuffer buffer;
+        private final ReadWriteLock latch;
+
+        public CachedPage(int cpid, ByteBuffer buffer) {
+            this.cpid = cpid;
+            this.buffer = buffer;
+            latch = new ReentrantReadWriteLock(true);
+        }
+
+        @Override
+        public ByteBuffer getBuffer() {
+            return buffer;
+        }
+
+        @Override
+        public Object getReplacementStrategyObject() {
+            // Do nothing.
+            return null;
+        }
+
+        @Override
+        public boolean pinIfGoodVictim() {
+            // Do nothing.
+            return false;
+        }
+
+        @Override
+        public int getCachedPageId() {
+            return cpid;
+        }
+
+        @Override
+        public void acquireReadLatch() {
+            latch.readLock().lock();
+        }
+
+        @Override
+        public void acquireWriteLatch() {
+            latch.writeLock().lock();
+        }
+
+        @Override
+        public void releaseReadLatch() {
+            latch.readLock().unlock();
+        }
+
+        @Override
+        public void releaseWriteLatch() {
+            latch.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public void force(int fileId, boolean metadata) throws HyracksDataException {
+    }
+
+    @Override
+    public void flushDirtyPage(ICachedPage page) throws HyracksDataException {
+    }
+
+    public IFileMapProvider getFileMapProvider() {
+        return fileMapManager;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryFreePageManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryFreePageManager.java
new file mode 100644
index 0000000..c601a9b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/freepage/InMemoryFreePageManager.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.freepage;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+
+public class InMemoryFreePageManager implements IInMemoryFreePageManager {
+    protected final int capacity;
+    protected final AtomicInteger currentPageId = new AtomicInteger();
+    protected final ITreeIndexMetaDataFrameFactory metaDataFrameFactory;
+
+    public InMemoryFreePageManager(int capacity, ITreeIndexMetaDataFrameFactory metaDataFrameFactory) {
+        // We start the currentPageId from 1, because the BTree uses
+        // the first page as metadata page, and the second page as root page.
+        // (when returning free pages we first increment, then get)
+        currentPageId.set(1);
+        this.capacity = capacity;
+        this.metaDataFrameFactory = metaDataFrameFactory;
+    }
+
+    @Override
+    public int getFreePage(ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException {
+        // The very first call returns page id 2 because the BTree uses
+        // the first page as metadata page, and the second page as root page.
+        return currentPageId.incrementAndGet();
+    }
+
+    @Override
+    public int getMaxPage(ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException {
+        return currentPageId.get();
+    }
+
+    @Override
+    public void init(ITreeIndexMetaDataFrame metaFrame, int currentMaxPage) throws HyracksDataException {
+        currentPageId.set(1);
+    }
+
+    @Override
+    public ITreeIndexMetaDataFrameFactory getMetaDataFrameFactory() {
+        return metaDataFrameFactory;
+    }
+
+    public int getCapacity() {
+        return capacity - 2;
+    }
+
+    public void reset() {
+        currentPageId.set(1);
+    }
+
+    public boolean isFull() {
+        return currentPageId.get() >= capacity;
+    }
+
+    @Override
+    public void addFreePage(ITreeIndexMetaDataFrame metaFrame, int freePage) throws HyracksDataException {
+    }
+
+    @Override
+    public byte getMetaPageLevelIndicator() {
+        return 0;
+    }
+
+    @Override
+    public byte getFreePageLevelIndicator() {
+        return 0;
+    }
+
+    @Override
+    public boolean isMetaPage(ITreeIndexMetaDataFrame metaFrame) {
+        return false;
+    }
+
+    @Override
+    public boolean isFreePage(ITreeIndexMetaDataFrame metaFrame) {
+        return false;
+    }
+
+    @Override
+    public int getFirstMetadataPage() {
+        // Method doesn't make sense for this free page manager.
+        return -1;
+    }
+
+    @Override
+    public void open(int fileId) {
+        // Method doesn't make sense for this free page manager.
+    }
+
+    @Override
+    public void close() {
+        // Method doesn't make sense for this free page manager.
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractImmutableLSMComponent.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractImmutableLSMComponent.java
new file mode 100644
index 0000000..b6fc2f7
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractImmutableLSMComponent.java
@@ -0,0 +1,69 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+
+public abstract class AbstractImmutableLSMComponent implements ILSMComponent {
+
+    private ComponentState state;
+    private int readerCount;
+
+    private enum ComponentState {
+        READABLE,
+        READABLE_MERGING,
+        KILLED
+    }
+
+    public AbstractImmutableLSMComponent() {
+        state = ComponentState.READABLE;
+        readerCount = 0;
+    }
+
+    @Override
+    public synchronized boolean threadEnter(LSMOperationType opType) {
+        if (state == ComponentState.KILLED) {
+            return false;
+        }
+
+        switch (opType) {
+            case MODIFICATION:
+            case SEARCH:
+                readerCount++;
+                break;
+            case MERGE:
+                if (state == ComponentState.READABLE_MERGING) {
+                    return false;
+                }
+                state = ComponentState.READABLE_MERGING;
+                readerCount++;
+                break;
+            default:
+                throw new UnsupportedOperationException("Unsupported operation " + opType);
+        }
+        return true;
+    }
+
+    @Override
+    public synchronized void threadExit(LSMOperationType opType, boolean failedOperation) throws HyracksDataException {
+        switch (opType) {
+            case MERGE:
+                if (failedOperation) {
+                    state = ComponentState.READABLE;
+                }
+            case MODIFICATION:
+            case SEARCH:
+                readerCount--;
+
+                if (readerCount == 0 && state == ComponentState.READABLE_MERGING) {
+                    destroy();
+                    state = ComponentState.KILLED;
+                }
+                break;
+            default:
+                throw new UnsupportedOperationException("Unsupported operation " + opType);
+        }
+    }
+
+    protected abstract void destroy() throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
new file mode 100644
index 0000000..0c6b9ab
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -0,0 +1,194 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+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.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public abstract class AbstractLSMIndex implements ILSMIndexInternal {
+    protected final static double MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE = 0.1;
+
+    protected final ILSMHarness lsmHarness;
+
+    protected final ILSMIOOperationScheduler ioScheduler;
+    protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+
+    // In-memory components.   
+    protected final IInMemoryFreePageManager memFreePageManager;
+
+    // On-disk components.    
+    protected final IBufferCache diskBufferCache;
+    protected final ILSMIndexFileManager fileManager;
+    protected final IFileMapProvider diskFileMapProvider;
+    protected final AtomicReference<List<ILSMComponent>> componentsRef;
+
+    protected boolean isActivated;
+
+    private boolean needsFlush = false;
+
+    public AbstractLSMIndex(IInMemoryFreePageManager memFreePageManager, IBufferCache diskBufferCache,
+            ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        this.memFreePageManager = memFreePageManager;
+        this.diskBufferCache = diskBufferCache;
+        this.diskFileMapProvider = diskFileMapProvider;
+        this.fileManager = fileManager;
+        this.ioScheduler = ioScheduler;
+        this.ioOpCallbackProvider = ioOpCallbackProvider;
+        ILSMOperationTracker opTracker = opTrackerFactory.createOperationTracker(this);
+        lsmHarness = new LSMHarness(this, mergePolicy, opTracker);
+        isActivated = false;
+        componentsRef = new AtomicReference<List<ILSMComponent>>();
+        componentsRef.set(new LinkedList<ILSMComponent>());
+    }
+
+    protected void forceFlushDirtyPages(ITreeIndex treeIndex) throws HyracksDataException {
+        int fileId = treeIndex.getFileId();
+        IBufferCache bufferCache = treeIndex.getBufferCache();
+        // Flush all dirty pages of the tree. 
+        // By default, metadata and data are flushed asynchronously in the buffercache.
+        // This means that the flush issues writes to the OS, but the data may still lie in filesystem buffers.
+        ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
+        int startPage = 0;
+        int maxPage = treeIndex.getFreePageManager().getMaxPage(metadataFrame);
+        forceFlushDirtyPages(bufferCache, fileId, startPage, maxPage);
+    }
+
+    protected void forceFlushDirtyPages(IBufferCache bufferCache, int fileId, int startPageId, int endPageId)
+            throws HyracksDataException {
+        for (int i = startPageId; i <= endPageId; i++) {
+            ICachedPage page = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, i));
+            // If tryPin returns null, it means the page is not cached, and therefore cannot be dirty.
+            if (page == null) {
+                continue;
+            }
+            try {
+                bufferCache.flushDirtyPage(page);
+            } finally {
+                bufferCache.unpin(page);
+            }
+        }
+        // Forces all pages of given file to disk. This guarantees the data makes it to disk.
+        bufferCache.force(fileId, true);
+    }
+
+    protected void markAsValidInternal(ITreeIndex treeIndex) throws HyracksDataException {
+        int fileId = treeIndex.getFileId();
+        IBufferCache bufferCache = treeIndex.getBufferCache();
+        ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
+        // Mark the component as a valid component by flushing the metadata page to disk
+        int metadataPageId = treeIndex.getFreePageManager().getFirstMetadataPage();
+        ICachedPage metadataPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId), false);
+        metadataPage.acquireWriteLatch();
+        try {
+            metadataFrame.setPage(metadataPage);
+            metadataFrame.setValid(true);
+
+            // Flush the single modified page to disk.
+            bufferCache.flushDirtyPage(metadataPage);
+
+            // Force modified metadata page to disk.
+            bufferCache.force(fileId, true);
+        } finally {
+            metadataPage.releaseWriteLatch();
+            bufferCache.unpin(metadataPage);
+        }
+    }
+
+    @Override
+    public void addComponent(ILSMComponent c) {
+        List<ILSMComponent> oldList = componentsRef.get();
+        List<ILSMComponent> newList = new ArrayList<ILSMComponent>();
+        newList.add(c);
+        for (ILSMComponent oc : oldList) {
+            newList.add(oc);
+        }
+        componentsRef.set(newList);
+    }
+
+    @Override
+    public void subsumeMergedComponents(ILSMComponent newComponent, List<ILSMComponent> mergedComponents) {
+        List<ILSMComponent> oldList = componentsRef.get();
+        List<ILSMComponent> newList = new ArrayList<ILSMComponent>();
+        int swapIndex = oldList.indexOf(mergedComponents.get(0));
+        int swapSize = mergedComponents.size();
+        for (int i = 0; i < oldList.size(); i++) {
+            if (i < swapIndex || i >= swapIndex + swapSize) {
+                newList.add(oldList.get(i));
+            } else if (i == swapIndex) {
+                newList.add(newComponent);
+            }
+        }
+        componentsRef.set(newList);
+    }
+
+    @Override
+    public IInMemoryFreePageManager getInMemoryFreePageManager() {
+        return memFreePageManager;
+    }
+
+    @Override
+    public List<ILSMComponent> getImmutableComponents() {
+        return componentsRef.get();
+    }
+
+    @Override
+    public void setFlushStatus(ILSMIndex index, boolean needsFlush) {
+        this.needsFlush = needsFlush;
+    }
+
+    @Override
+    public boolean getFlushStatus(ILSMIndex index) {
+        return needsFlush;
+    }
+
+    @Override
+    public ILSMOperationTracker getOperationTracker() {
+        return lsmHarness.getOperationTracker();
+    }
+
+    @Override
+    public ILSMIOOperationScheduler getIOScheduler() {
+        return ioScheduler;
+    }
+
+    @Override
+    public IBufferCache getBufferCache() {
+        return diskBufferCache;
+    }
+}
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
new file mode 100644
index 0000000..a84f8c9
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
@@ -0,0 +1,295 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.text.Format;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.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;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public abstract class AbstractLSMIndexFileManager implements ILSMIndexFileManager {
+
+    protected static final String SPLIT_STRING = "_";
+    protected static final String BLOOM_FILTER_STRING = "f";
+
+    // Use all IODevices registered in ioManager in a round-robin fashion to choose
+    // where to flush and merge
+    protected final IIOManager ioManager;
+    protected final IFileMapProvider fileMapProvider;
+
+    // baseDir should reflect dataset name and partition name.
+    protected String baseDir;
+    protected final Format formatter = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss-SSS");
+    protected final Comparator<String> cmp = new FileNameComparator();
+    protected final Comparator<ComparableFileName> recencyCmp = new RecencyComparator();
+
+    protected final TreeIndexFactory<? extends ITreeIndex> treeFactory;
+
+    // The current index for the round-robin file assignment
+    private int ioDeviceIndex = 0;
+
+    public AbstractLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> treeFactory, int startIODeviceIndex) {
+        this.baseDir = file.getFile().getPath();
+        if (!baseDir.endsWith(System.getProperty("file.separator"))) {
+            baseDir += System.getProperty("file.separator");
+        }
+        this.fileMapProvider = fileMapProvider;
+        this.ioManager = ioManager;
+        this.treeFactory = treeFactory;
+        ioDeviceIndex = startIODeviceIndex % ioManager.getIODevices().size();
+    }
+
+    private static FilenameFilter fileNameFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".");
+        }
+    };
+
+    protected boolean isValidTreeIndex(ITreeIndex treeIndex) throws HyracksDataException {
+        IBufferCache bufferCache = treeIndex.getBufferCache();
+        treeIndex.activate();
+        try {
+            int metadataPage = treeIndex.getFreePageManager().getFirstMetadataPage();
+            ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory()
+                    .createFrame();
+            ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(treeIndex.getFileId(), metadataPage),
+                    false);
+            page.acquireReadLatch();
+            try {
+                metadataFrame.setPage(page);
+                return metadataFrame.isValid();
+            } finally {
+                page.releaseReadLatch();
+                bufferCache.unpin(page);
+            }
+        } finally {
+            treeIndex.deactivate();
+        }
+    }
+
+    protected void cleanupAndGetValidFilesInternal(IODeviceHandle dev, FilenameFilter filter,
+            TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<ComparableFileName> allFiles)
+            throws HyracksDataException, IndexException {
+        File dir = new File(dev.getPath(), baseDir);
+        String[] files = dir.list(filter);
+        for (String fileName : files) {
+            File file = new File(dir.getPath() + File.separator + fileName);
+            FileReference fileRef = new FileReference(file);
+            if (treeFactory == null || isValidTreeIndex(treeFactory.createIndexInstance(fileRef))) {
+                allFiles.add(new ComparableFileName(fileRef));
+            } else {
+                file.delete();
+            }
+        }
+    }
+
+    @Override
+    public void createDirs() {
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            File f = new File(dev.getPath(), baseDir);
+            f.mkdirs();
+        }
+    }
+
+    @Override
+    public void deleteDirs() {
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            File f = new File(dev.getPath(), baseDir);
+            delete(f);
+        }
+    }
+
+    private void delete(File f) {
+        if (f.isDirectory()) {
+            for (File c : f.listFiles()) {
+                delete(c);
+            }
+        }
+        f.delete();
+    }
+
+    protected static FilenameFilter bloomFilterFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".") && name.endsWith(BLOOM_FILTER_STRING);
+        }
+    };
+
+    protected FileReference createFlushFile(String relFlushFileName) {
+        // Assigns new files to I/O devices in round-robin fashion.
+        IODeviceHandle dev = ioManager.getIODevices().get(ioDeviceIndex);
+        ioDeviceIndex = (ioDeviceIndex + 1) % ioManager.getIODevices().size();
+        return dev.createFileReference(relFlushFileName);
+    }
+
+    protected FileReference createMergeFile(String relMergeFileName) {
+        return createFlushFile(relMergeFileName);
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelFlushFileReference() {
+        Date date = new Date();
+        String ts = formatter.format(date);
+        // Begin timestamp and end timestamp are identical since it is a flush
+        return new LSMComponentFileReferences(createFlushFile(baseDir + ts + SPLIT_STRING + ts), null, null);
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
+            throws HyracksDataException {
+        String[] firstTimestampRange = firstFileName.split(SPLIT_STRING);
+        String[] lastTimestampRange = lastFileName.split(SPLIT_STRING);
+        // Get the range of timestamps by taking the earliest and the latest timestamps
+        return new LSMComponentFileReferences(createMergeFile(baseDir + firstTimestampRange[0] + SPLIT_STRING
+                + lastTimestampRange[1]), null, null);
+    }
+
+    @Override
+    public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException, IndexException {
+        List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
+        ArrayList<ComparableFileName> allFiles = new ArrayList<ComparableFileName>();
+
+        // Gather files from all IODeviceHandles 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).
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            cleanupAndGetValidFilesInternal(dev, fileNameFilter, treeFactory, allFiles);
+        }
+
+        if (allFiles.isEmpty()) {
+            return validFiles;
+        }
+
+        if (allFiles.size() == 1) {
+            validFiles.add(new LSMComponentFileReferences(allFiles.get(0).fileRef, null, null));
+            return validFiles;
+        }
+
+        // Sorts files names from earliest to latest timestamp.
+        Collections.sort(allFiles);
+
+        List<ComparableFileName> validComparableFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName last = allFiles.get(0);
+        validComparableFiles.add(last);
+        for (int i = 1; i < allFiles.size(); i++) {
+            ComparableFileName current = allFiles.get(i);
+            // The current start timestamp is greater than last stop timestamp so current is valid.
+            if (current.interval[0].compareTo(last.interval[1]) > 0) {
+                validComparableFiles.add(current);
+                last = current;
+            } else if (current.interval[0].compareTo(last.interval[0]) >= 0
+                    && current.interval[1].compareTo(last.interval[1]) <= 0) {
+                // The current file is completely contained in the interval of the 
+                // last file. Thus the last file must contain at least as much information 
+                // as the current file, so delete the current file.
+                current.fileRef.delete();
+            } else {
+                // This scenario should not be possible since timestamps are monotonically increasing.
+                throw new HyracksDataException("Found LSM files with overlapping timestamp intervals, "
+                        + "but the intervals were not contained by another file.");
+            }
+        }
+
+        // Sort valid files in reverse lexicographical order, such that newer files come first.
+        Collections.sort(validComparableFiles, recencyCmp);
+        for (ComparableFileName cmpFileName : validComparableFiles) {
+            validFiles.add(new LSMComponentFileReferences(cmpFileName.fileRef, null, null));
+        }
+
+        return validFiles;
+    }
+
+    @Override
+    public Comparator<String> getFileNameComparator() {
+        return cmp;
+    }
+
+    /**
+     * Sorts strings in reverse lexicographical order. The way we construct the
+     * file names above guarantees that:
+     * 1. Flushed files sort lower than merged files
+     * 2. Flushed files are sorted from newest to oldest (based on the timestamp
+     * string)
+     */
+    private class FileNameComparator implements Comparator<String> {
+        @Override
+        public int compare(String a, String b) {
+            // Consciously ignoring locale.
+            return -a.compareTo(b);
+        }
+    }
+
+    @Override
+    public String getBaseDir() {
+        return baseDir;
+    }
+
+    protected class ComparableFileName implements Comparable<ComparableFileName> {
+        public final FileReference fileRef;
+        public final String fullPath;
+        public final String fileName;
+
+        // Timestamp interval.
+        public final String[] interval;
+
+        public ComparableFileName(FileReference fileRef) {
+            this.fileRef = fileRef;
+            this.fullPath = fileRef.getFile().getAbsolutePath();
+            this.fileName = fileRef.getFile().getName();
+            interval = fileName.split(SPLIT_STRING);
+        }
+
+        @Override
+        public int compareTo(ComparableFileName b) {
+            int startCmp = interval[0].compareTo(b.interval[0]);
+            if (startCmp != 0) {
+                return startCmp;
+            }
+            return b.interval[1].compareTo(interval[1]);
+        }
+    }
+
+    private class RecencyComparator implements Comparator<ComparableFileName> {
+        @Override
+        public int compare(ComparableFileName a, ComparableFileName b) {
+            int cmp = -a.interval[0].compareTo(b.interval[0]);
+            if (cmp != 0) {
+                return cmp;
+            }
+            return -a.interval[1].compareTo(b.interval[1]);
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMutableLSMComponent.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMutableLSMComponent.java
new file mode 100644
index 0000000..1a6636a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractMutableLSMComponent.java
@@ -0,0 +1,104 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+
+public abstract class AbstractMutableLSMComponent implements ILSMComponent {
+
+    private int readerCount;
+    private int writerCount;
+    private ComponentState state;
+
+    private enum ComponentState {
+        READABLE_WRITABLE,
+        READABLE_UNWRITABLE,
+        READABLE_UNWRITABLE_FLUSHING,
+        UNREADABLE_UNWRITABLE
+    }
+
+    public AbstractMutableLSMComponent() {
+        readerCount = 0;
+        writerCount = 0;
+        state = ComponentState.READABLE_WRITABLE;
+    }
+
+    @Override
+    public synchronized boolean threadEnter(LSMOperationType opType) throws InterruptedException {
+        switch (opType) {
+            case FORCE_MODIFICATION:
+                if (state != ComponentState.READABLE_WRITABLE && state != ComponentState.READABLE_UNWRITABLE) {
+                    return false;
+                }
+                writerCount++;
+                break;
+            case MODIFICATION:
+                if (state != ComponentState.READABLE_WRITABLE) {
+                    return false;
+                }
+                writerCount++;
+                break;
+            case SEARCH:
+                if (state == ComponentState.UNREADABLE_UNWRITABLE) {
+                    return false;
+                }
+                readerCount++;
+                break;
+            case FLUSH:
+                if (state == ComponentState.READABLE_UNWRITABLE_FLUSHING
+                        || state == ComponentState.UNREADABLE_UNWRITABLE) {
+                    return false;
+                }
+
+                state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
+                while (writerCount > 0) {
+                    wait();
+                }
+                readerCount++;
+                break;
+            default:
+                throw new UnsupportedOperationException("Unsupported operation " + opType);
+        }
+        return true;
+    }
+
+    @Override
+    public synchronized void threadExit(LSMOperationType opType, boolean failedOperation) throws HyracksDataException {
+        switch (opType) {
+            case FORCE_MODIFICATION:
+            case MODIFICATION:
+                writerCount--;
+                if (state == ComponentState.READABLE_WRITABLE && isFull()) {
+                    state = ComponentState.READABLE_UNWRITABLE;
+                }
+                break;
+            case SEARCH:
+                readerCount--;
+                if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
+                    reset();
+                    state = ComponentState.READABLE_WRITABLE;
+                } else if (state == ComponentState.READABLE_WRITABLE && isFull()) {
+                    state = ComponentState.READABLE_UNWRITABLE;
+                }
+                break;
+            case FLUSH:
+                if (failedOperation) {
+                    state = isFull() ? ComponentState.READABLE_UNWRITABLE : ComponentState.READABLE_WRITABLE;
+                }
+                readerCount--;
+                if (readerCount == 0) {
+                    reset();
+                    state = ComponentState.READABLE_WRITABLE;
+                } else if (state == ComponentState.READABLE_UNWRITABLE_FLUSHING) {
+                    state = ComponentState.UNREADABLE_UNWRITABLE;
+                }
+                break;
+            default:
+                throw new UnsupportedOperationException("Unsupported operation " + opType);
+        }
+        notifyAll();
+    }
+
+    protected abstract boolean isFull();
+
+    protected abstract void reset() throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BTreeFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BTreeFactory.java
new file mode 100644
index 0000000..008c418
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BTreeFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManagerFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class BTreeFactory extends TreeIndexFactory<BTree> {
+
+    public BTreeFactory(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            IFreePageManagerFactory freePageManagerFactory, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount) {
+        super(bufferCache, fileMapProvider, freePageManagerFactory, interiorFrameFactory, leafFrameFactory,
+                cmpFactories, fieldCount);
+    }
+
+    @Override
+    public BTree createIndexInstance(FileReference file) throws IndexException {
+        return new BTree(bufferCache, fileMapProvider, freePageManagerFactory.createFreePageManager(),
+                interiorFrameFactory, leafFrameFactory, cmpFactories, fieldCount, file);
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
new file mode 100644
index 0000000..34e1f0d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
@@ -0,0 +1,43 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+
+public class BlockingIOOperationCallbackWrapper implements ILSMIOOperationCallback {
+
+    private boolean notified = false;
+
+    private final ILSMIOOperationCallback wrappedCallback;
+
+    public BlockingIOOperationCallbackWrapper(ILSMIOOperationCallback callback) {
+        this.wrappedCallback = callback;
+    }
+
+    public synchronized void waitForIO() throws InterruptedException {
+        if (!notified) {
+            this.wait();
+        }
+        notified = false;
+    }
+
+    @Override
+    public void beforeOperation() throws HyracksDataException {
+        wrappedCallback.beforeOperation();
+    }
+
+    @Override
+    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+            throws HyracksDataException {
+        wrappedCallback.afterOperation(oldComponents, newComponent);
+    }
+
+    @Override
+    public synchronized void afterFinalize(ILSMComponent newComponent) throws HyracksDataException {
+        wrappedCallback.afterFinalize(newComponent);
+        this.notifyAll();
+        notified = true;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BloomFilterAwareBTreePointSearchCursor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BloomFilterAwareBTreePointSearchCursor.java
new file mode 100644
index 0000000..af08bdb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BloomFilterAwareBTreePointSearchCursor.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+
+public class BloomFilterAwareBTreePointSearchCursor extends BTreeRangeSearchCursor {
+    private final BloomFilter bloomFilter;
+    private long[] hashes = new long[2];
+
+    public BloomFilterAwareBTreePointSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes,
+            BloomFilter bloomFilter) {
+        super(frame, exclusiveLatchNodes);
+        this.bloomFilter = bloomFilter;
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException {
+        if (bloomFilter.contains(lowKey, hashes)) {
+            return super.hasNext();
+        }
+        return false;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
new file mode 100644
index 0000000..1c72abf
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+
+public class ConstantMergePolicy implements ILSMMergePolicy {
+
+    private final int threshold;
+
+    public ConstantMergePolicy(int threshold) {
+        this.threshold = threshold;
+    }
+
+    @Override
+    public void diskComponentAdded(final ILSMIndex index, int totalNumDiskComponents) throws HyracksDataException,
+            IndexException {
+        if (totalNumDiskComponents >= threshold) {
+            ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyProvider.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyProvider.java
new file mode 100644
index 0000000..b404c9b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ConstantMergePolicyProvider.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+
+public class ConstantMergePolicyProvider implements ILSMMergePolicyProvider {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int threshold;
+
+    public ConstantMergePolicyProvider(int threshold) {
+        this.threshold = threshold;
+    }
+
+    @Override
+    public ILSMMergePolicy getMergePolicy(IHyracksTaskContext ctx) {
+        return new ConstantMergePolicy(threshold);
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/IndexFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/IndexFactory.java
new file mode 100644
index 0000000..3feaecf
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/IndexFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManagerFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public abstract class IndexFactory<T extends IIndex> {
+
+    protected final IBufferCache bufferCache;
+    protected final IFileMapProvider fileMapProvider;
+    protected final IFreePageManagerFactory freePageManagerFactory;
+
+    public IndexFactory(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            IFreePageManagerFactory freePageManagerFactory) {
+        this.bufferCache = bufferCache;
+        this.fileMapProvider = fileMapProvider;
+        this.freePageManagerFactory = freePageManagerFactory;
+    }
+
+    public abstract T createIndexInstance(FileReference file) throws IndexException;
+
+    public IBufferCache getBufferCache() {
+        return bufferCache;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMComponentFileReferences.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMComponentFileReferences.java
new file mode 100644
index 0000000..019dca4
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMComponentFileReferences.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.io.FileReference;
+
+public final class LSMComponentFileReferences {
+
+    // The FileReference for the index that is used for inserting records of the component. For instance, this will be the FileReference of the RTree in one component of the LSM-RTree.
+    private final FileReference insertIndexFileReference;
+    // This FileReference for the delete index (if any). For example, this will be the the FileReference of the buddy BTree in one component of the LSM-RTree.
+    private final FileReference deleteIndexFileReference;
+
+    // This FileReference for the bloom filter (if any). 
+    private final FileReference bloomFilterFileReference;
+
+    public LSMComponentFileReferences(FileReference insertIndexFileReference, FileReference deleteIndexFileReference,
+            FileReference bloomFilterFileReference) {
+        this.insertIndexFileReference = insertIndexFileReference;
+        this.deleteIndexFileReference = deleteIndexFileReference;
+        this.bloomFilterFileReference = bloomFilterFileReference;
+    }
+
+    public FileReference getInsertIndexFileReference() {
+        return insertIndexFileReference;
+    }
+
+    public FileReference getDeleteIndexFileReference() {
+        return deleteIndexFileReference;
+    }
+
+    public FileReference getBloomFilterFileReference() {
+        return bloomFilterFileReference;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMComponentState.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMComponentState.java
new file mode 100644
index 0000000..e554a6e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMComponentState.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+public enum LSMComponentState {
+    FLUSHING,
+    MERGING,
+    DONE_FLUSHING,
+    DONE_MERGING
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
new file mode 100644
index 0000000..4a140b4
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -0,0 +1,231 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+
+public class LSMHarness implements ILSMHarness {
+    private final ILSMIndexInternal lsmIndex;
+    private final ILSMMergePolicy mergePolicy;
+    private final ILSMOperationTracker opTracker;
+
+    public LSMHarness(ILSMIndexInternal lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker) {
+        this.lsmIndex = lsmIndex;
+        this.opTracker = opTracker;
+        this.mergePolicy = mergePolicy;
+    }
+
+    private void threadExit(ILSMIndexOperationContext opCtx, LSMOperationType opType) throws HyracksDataException {
+        if (!lsmIndex.getFlushStatus(lsmIndex) && lsmIndex.getInMemoryFreePageManager().isFull()) {
+            lsmIndex.setFlushStatus(lsmIndex, true);
+        }
+        opTracker.afterOperation(opType, opCtx.getSearchOperationCallback(), opCtx.getModificationCallback());
+    }
+
+    private boolean getAndEnterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, boolean tryOperation)
+            throws HyracksDataException {
+        int numEntered = 0;
+        boolean entranceSuccessful = false;
+        List<ILSMComponent> entered = new ArrayList<ILSMComponent>();
+
+        while (!entranceSuccessful) {
+            entered.clear();
+            lsmIndex.getOperationalComponents(ctx);
+            List<ILSMComponent> components = ctx.getComponentHolder();
+            try {
+                for (ILSMComponent c : components) {
+                    if (!c.threadEnter(opType)) {
+                        break;
+                    }
+                    numEntered++;
+                    entered.add(c);
+                }
+                entranceSuccessful = numEntered == components.size();
+            } catch (InterruptedException e) {
+                entranceSuccessful = false;
+                throw new HyracksDataException(e);
+            } finally {
+                if (!entranceSuccessful) {
+                    for (ILSMComponent c : components) {
+                        if (numEntered <= 0) {
+                            break;
+                        }
+                        c.threadExit(opType, true);
+                        numEntered--;
+                    }
+                }
+            }
+            if (tryOperation && !entranceSuccessful) {
+                return false;
+            }
+        }
+
+        opTracker.beforeOperation(opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
+        return true;
+    }
+
+    private void exitComponents(ILSMIndexOperationContext ctx, LSMOperationType opType, boolean failedOperation)
+            throws HyracksDataException {
+        try {
+            for (ILSMComponent c : ctx.getComponentHolder()) {
+                c.threadExit(opType, failedOperation);
+            }
+        } finally {
+            threadExit(ctx, opType);
+        }
+    }
+
+    @Override
+    public void forceModify(ILSMIndexOperationContext ctx, ITupleReference tuple) throws HyracksDataException,
+            IndexException {
+        LSMOperationType opType = LSMOperationType.FORCE_MODIFICATION;
+        modify(ctx, false, tuple, opType);
+    }
+
+    @Override
+    public boolean modify(ILSMIndexOperationContext ctx, boolean tryOperation, ITupleReference tuple)
+            throws HyracksDataException, IndexException {
+        LSMOperationType opType = LSMOperationType.MODIFICATION;
+        return modify(ctx, tryOperation, tuple, opType);
+    }
+
+    private boolean modify(ILSMIndexOperationContext ctx, boolean tryOperation, ITupleReference tuple,
+            LSMOperationType opType) throws HyracksDataException, IndexException {
+        if (!getAndEnterComponents(ctx, opType, tryOperation)) {
+            return false;
+        }
+        try {
+            lsmIndex.modify(ctx, tuple);
+        } finally {
+            exitComponents(ctx, opType, false);
+        }
+
+        return true;
+    }
+
+    @Override
+    public void search(ILSMIndexOperationContext ctx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException {
+        LSMOperationType opType = LSMOperationType.SEARCH;
+        getAndEnterComponents(ctx, opType, false);
+        try {
+            lsmIndex.search(ctx, cursor, pred);
+        } catch (HyracksDataException e) {
+            exitComponents(ctx, opType, true);
+            throw e;
+        } catch (IndexException e) {
+            exitComponents(ctx, opType, true);
+            throw e;
+        }
+    }
+
+    @Override
+    public void endSearch(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        if (ctx.getOperation() == IndexOperation.SEARCH) {
+            exitComponents(ctx, LSMOperationType.SEARCH, false);
+        }
+    }
+
+    @Override
+    public void noOp(ILSMIndexOperationContext ctx) throws HyracksDataException {
+        LSMOperationType opType = LSMOperationType.NOOP;
+        opTracker.beforeOperation(opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
+        threadExit(ctx, opType);
+    }
+
+    @Override
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException {
+        if (!getAndEnterComponents(ctx, LSMOperationType.FLUSH, true)) {
+            return;
+        }
+        lsmIndex.setFlushStatus(lsmIndex, false);
+        lsmIndex.scheduleFlush(ctx, callback);
+    }
+
+    @Override
+    public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException,
+            IndexException {
+        operation.getCallback().beforeOperation();
+        ILSMComponent newComponent = lsmIndex.flush(operation);
+        operation.getCallback().afterOperation(null, newComponent);
+        lsmIndex.markAsValid(newComponent);
+        operation.getCallback().afterFinalize(newComponent);
+
+        lsmIndex.addComponent(newComponent);
+        int numComponents = lsmIndex.getImmutableComponents().size();
+
+        mergePolicy.diskComponentAdded(lsmIndex, numComponents);
+        exitComponents(ctx, LSMOperationType.FLUSH, false);
+    }
+
+    @Override
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        LSMOperationType opType = LSMOperationType.MERGE;
+        if (!getAndEnterComponents(ctx, opType, false)) {
+            return;
+        }
+        if (ctx.getComponentHolder().size() > 1) {
+            lsmIndex.scheduleMerge(ctx, callback);
+        } else {
+            exitComponents(ctx, opType, true);
+        }
+    }
+
+    @Override
+    public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException,
+            IndexException {
+        List<ILSMComponent> mergedComponents = new ArrayList<ILSMComponent>();
+        operation.getCallback().beforeOperation();
+        ILSMComponent newComponent = lsmIndex.merge(mergedComponents, operation);
+        ctx.getComponentHolder().addAll(mergedComponents);
+        operation.getCallback().afterOperation(mergedComponents, newComponent);
+        lsmIndex.markAsValid(newComponent);
+        operation.getCallback().afterFinalize(newComponent);
+        lsmIndex.subsumeMergedComponents(newComponent, mergedComponents);
+        exitComponents(ctx, LSMOperationType.MERGE, false);
+    }
+
+    @Override
+    public void addBulkLoadedComponent(ILSMComponent c) throws HyracksDataException, IndexException {
+        lsmIndex.markAsValid(c);
+        lsmIndex.addComponent(c);
+        int numComponents = lsmIndex.getImmutableComponents().size();
+        mergePolicy.diskComponentAdded(lsmIndex, numComponents);
+    }
+
+    @Override
+    public ILSMOperationTracker getOperationTracker() {
+        return opTracker;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
new file mode 100644
index 0000000..7f08ba4
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -0,0 +1,211 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public abstract class LSMIndexSearchCursor implements ITreeIndexCursor {
+    protected PriorityQueueElement outputElement;
+    protected IIndexCursor[] rangeCursors;
+    protected PriorityQueue<PriorityQueueElement> outputPriorityQueue;
+    protected PriorityQueueComparator pqCmp;
+    protected MultiComparator cmp;
+    protected boolean needPush;
+    protected boolean includeMemComponent;
+    protected ILSMHarness lsmHarness;
+    protected final ILSMIndexOperationContext opCtx;
+
+    protected List<ILSMComponent> operationalComponents;
+
+    public LSMIndexSearchCursor(ILSMIndexOperationContext opCtx) {
+        this.opCtx = opCtx;
+        outputElement = null;
+        needPush = false;
+    }
+
+    public void initPriorityQueue() throws HyracksDataException, IndexException {
+        int pqInitSize = (rangeCursors.length > 0) ? rangeCursors.length : 1;
+        outputPriorityQueue = new PriorityQueue<PriorityQueueElement>(pqInitSize, pqCmp);
+        for (int i = 0; i < rangeCursors.length; i++) {
+            pushIntoPriorityQueue(new PriorityQueueElement(i));
+        }
+    }
+
+    public IIndexCursor getCursor(int cursorIndex) {
+        return rangeCursors[cursorIndex];
+    }
+
+    @Override
+    public void reset() throws HyracksDataException, IndexException {
+        outputElement = null;
+        needPush = false;
+
+        if (outputPriorityQueue != null) {
+            outputPriorityQueue.clear();
+        }
+
+        if (rangeCursors != null) {
+            for (int i = 0; i < rangeCursors.length; i++) {
+                rangeCursors[i].reset();
+            }
+        }
+        rangeCursors = null;
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        checkPriorityQueue();
+        return !outputPriorityQueue.isEmpty();
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        outputElement = outputPriorityQueue.poll();
+        needPush = true;
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        // do nothing
+        return null;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (lsmHarness != null) {
+            try {
+                outputPriorityQueue.clear();
+                for (int i = 0; i < rangeCursors.length; i++) {
+                    rangeCursors[i].close();
+                }
+                rangeCursors = null;
+            } finally {
+                lsmHarness.endSearch(opCtx);
+            }
+        }
+    }
+
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        // do nothing
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        // do nothing
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return outputElement.getTuple();
+    }
+
+    protected boolean pushIntoPriorityQueue(PriorityQueueElement e) throws HyracksDataException, IndexException {
+        int cursorIndex = e.getCursorIndex();
+        if (rangeCursors[cursorIndex].hasNext()) {
+            rangeCursors[cursorIndex].next();
+            e.reset(rangeCursors[cursorIndex].getTuple());
+            outputPriorityQueue.offer(e);
+            return true;
+        }
+        rangeCursors[cursorIndex].close();
+        return false;
+    }
+
+    protected boolean isDeleted(PriorityQueueElement checkElement) throws HyracksDataException, IndexException {
+        return ((ILSMTreeTupleReference) checkElement.getTuple()).isAntimatter();
+    }
+
+    abstract protected void checkPriorityQueue() throws HyracksDataException, IndexException;
+
+    @Override
+    public boolean exclusiveLatchNodes() {
+        return false;
+    }
+
+    public class PriorityQueueElement {
+        private ITupleReference tuple;
+        private final int cursorIndex;
+
+        public PriorityQueueElement(int cursorIndex) {
+            tuple = null;
+            this.cursorIndex = cursorIndex;
+        }
+
+        public ITupleReference getTuple() {
+            return tuple;
+        }
+
+        public int getCursorIndex() {
+            return cursorIndex;
+        }
+
+        public void reset(ITupleReference tuple) {
+            this.tuple = tuple;
+        }
+    }
+
+    public class PriorityQueueComparator implements Comparator<PriorityQueueElement> {
+
+        protected final MultiComparator cmp;
+
+        public PriorityQueueComparator(MultiComparator cmp) {
+            this.cmp = cmp;
+        }
+
+        @Override
+        public int compare(PriorityQueueElement elementA, PriorityQueueElement elementB) {
+            int result = cmp.compare(elementA.getTuple(), elementB.getTuple());
+            if (result != 0) {
+                return result;
+            }
+            if (elementA.getCursorIndex() > elementB.getCursorIndex()) {
+                return 1;
+            } else {
+                return -1;
+            }
+        }
+
+        public MultiComparator getMultiComparator() {
+            return cmp;
+        }
+    }
+
+    protected void setPriorityQueueComparator() {
+        if (pqCmp == null || cmp != pqCmp.getMultiComparator()) {
+            pqCmp = new PriorityQueueComparator(cmp);
+        }
+    }
+
+    protected int compare(MultiComparator cmp, ITupleReference tupleA, ITupleReference tupleB) {
+        return cmp.compare(tupleA, tupleB);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMOperationType.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMOperationType.java
new file mode 100644
index 0000000..981cefe
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMOperationType.java
@@ -0,0 +1,10 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+public enum LSMOperationType {
+    SEARCH,
+    MODIFICATION,
+    FORCE_MODIFICATION,
+    FLUSH,
+    MERGE,
+    NOOP
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
new file mode 100644
index 0000000..7cc29a5
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
@@ -0,0 +1,146 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public abstract class LSMTreeIndexAccessor implements ILSMIndexAccessorInternal {
+    protected ILSMHarness lsmHarness;
+    protected ILSMIndexOperationContext ctx;
+
+    public LSMTreeIndexAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx) {
+        this.lsmHarness = lsmHarness;
+        this.ctx = ctx;
+    }
+
+    @Override
+    public void insert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.INSERT);
+        lsmHarness.modify(ctx, false, tuple);
+    }
+
+    @Override
+    public void update(ITupleReference tuple) throws HyracksDataException, IndexException {
+        // Update is the same as insert.
+        ctx.setOperation(IndexOperation.UPDATE);
+        lsmHarness.modify(ctx, false, tuple);
+    }
+
+    @Override
+    public void delete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.DELETE);
+        lsmHarness.modify(ctx, false, tuple);
+    }
+
+    @Override
+    public void upsert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.UPSERT);
+        lsmHarness.modify(ctx, false, tuple);
+    }
+
+    @Override
+    public boolean tryInsert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.INSERT);
+        return lsmHarness.modify(ctx, true, tuple);
+    }
+
+    @Override
+    public boolean tryDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.DELETE);
+        return lsmHarness.modify(ctx, true, tuple);
+    }
+
+    @Override
+    public boolean tryUpdate(ITupleReference tuple) throws HyracksDataException, IndexException {
+        // Update is the same as insert.
+        ctx.setOperation(IndexOperation.UPDATE);
+        return lsmHarness.modify(ctx, true, tuple);
+    }
+
+    @Override
+    public boolean tryUpsert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.UPSERT);
+        return lsmHarness.modify(ctx, true, tuple);
+    }
+
+    @Override
+    public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.SEARCH);
+        lsmHarness.search(ctx, cursor, searchPred);
+    }
+
+    @Override
+    public void flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        lsmHarness.flush(ctx, operation);
+    }
+
+    @Override
+    public void merge(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.MERGE);
+        lsmHarness.merge(ctx, operation);
+    }
+
+    @Override
+    public void physicalDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.PHYSICALDELETE);
+        lsmHarness.modify(ctx, false, tuple);
+    }
+
+    @Override
+    public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+        ctx.setOperation(IndexOperation.FLUSH);
+        lsmHarness.scheduleFlush(ctx, callback);
+    }
+
+    @Override
+    public void scheduleMerge(ILSMIOOperationCallback callback) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.MERGE);
+        lsmHarness.scheduleMerge(ctx, callback);
+    }
+
+    @Override
+    public void noOp() throws HyracksDataException {
+        lsmHarness.noOp(ctx);
+    }
+
+    @Override
+    public void forcePhysicalDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.PHYSICALDELETE);
+        lsmHarness.forceModify(ctx, tuple);
+    }
+
+    @Override
+    public void forceInsert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.INSERT);
+        lsmHarness.forceModify(ctx, tuple);
+    }
+
+    @Override
+    public void forceDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.DELETE);
+        lsmHarness.forceModify(ctx, tuple);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoMergePolicy.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoMergePolicy.java
new file mode 100644
index 0000000..5d36c09
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoMergePolicy.java
@@ -0,0 +1,14 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+
+public enum NoMergePolicy implements ILSMMergePolicy {
+    INSTANCE;
+
+    @Override
+    public void diskComponentAdded(ILSMIndex index, int totalNumDiskComponents) {
+        // Do nothing
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
new file mode 100644
index 0000000..b123b30
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
@@ -0,0 +1,34 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+
+public enum NoOpIOOperationCallback implements ILSMIOOperationCallback, ILSMIOOperationCallbackProvider {
+    INSTANCE;
+
+    @Override
+    public void beforeOperation() throws HyracksDataException {
+        // Do nothing.
+    }
+
+    @Override
+    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+            throws HyracksDataException {
+        // Do nothing.
+    }
+
+    @Override
+    public void afterFinalize(ILSMComponent newComponent) throws HyracksDataException {
+        // Do nothing.
+    }
+
+    @Override
+    public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index) {
+        return INSTANCE;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
new file mode 100644
index 0000000..97ec50e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpOperationTrackerFactory.java
@@ -0,0 +1,48 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+
+/**
+ * Operation tracker that does nothing.
+ * WARNING: This op tracker should only be used for specific testing purposes.
+ * It is assumed than an op tracker cooperates with an lsm index to synchronize flushes with
+ * regular operations, and this implementation does no such tracking at all.
+ */
+public class NoOpOperationTrackerFactory implements ILSMOperationTrackerFactory {
+    private static final long serialVersionUID = 1L;
+
+    public static NoOpOperationTrackerFactory INSTANCE = new NoOpOperationTrackerFactory();
+
+    @Override
+    public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
+        return new ILSMOperationTracker() {
+
+            @Override
+            public void completeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+                    IModificationOperationCallback modificationCallback) throws HyracksDataException {
+                // Do nothing.
+            }
+
+            @Override
+            public void beforeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+                    IModificationOperationCallback modificationCallback) throws HyracksDataException {
+            }
+
+            @Override
+            public void afterOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+                    IModificationOperationCallback modificationCallback) throws HyracksDataException {
+                // Do nothing.                        
+            }
+        };
+    }
+
+    // Enforce singleton.
+    private NoOpOperationTrackerFactory() {
+    }
+
+};
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
new file mode 100644
index 0000000..9bbd394
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
@@ -0,0 +1,19 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+
+public enum SynchronousScheduler implements ILSMIOOperationScheduler {
+    INSTANCE;
+
+    @Override
+    public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
+        try {
+            operation.perform();
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
new file mode 100644
index 0000000..72d9d1d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+
+public enum SynchronousSchedulerProvider implements ILSMIOOperationSchedulerProvider {
+    INSTANCE;
+
+    @Override
+    public ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx) {
+        return SynchronousScheduler.INSTANCE;
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerFactory.java
new file mode 100644
index 0000000..3b4b00f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingOperationTrackerFactory.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+
+public class ThreadCountingOperationTrackerFactory implements ILSMOperationTrackerFactory {
+
+    private static final long serialVersionUID = 1L;
+    
+    public static ThreadCountingOperationTrackerFactory INSTANCE = new ThreadCountingOperationTrackerFactory(); 
+    
+    @Override
+    public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
+        return new ThreadCountingTracker(index);
+    }
+
+    // Enforce singleton.
+    private ThreadCountingOperationTrackerFactory() {
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
new file mode 100644
index 0000000..7fee06e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
@@ -0,0 +1,49 @@
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+
+public class ThreadCountingTracker implements ILSMOperationTracker {
+    private final AtomicInteger threadRefCount;
+    private final ILSMIndex index;
+
+    public ThreadCountingTracker(ILSMIndex index) {
+        this.index = index;
+        this.threadRefCount = new AtomicInteger();
+    }
+
+    @Override
+    public void beforeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        if (opType == LSMOperationType.MODIFICATION) {
+            threadRefCount.incrementAndGet();
+        }
+    }
+
+    @Override
+    public void afterOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        // The operation is considered inactive, immediately after leaving the index.
+        completeOperation(opType, searchCallback, modificationCallback);
+    }
+
+    @Override
+    public void completeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        // Flush will only be handled by last exiting thread.
+        if (opType == LSMOperationType.MODIFICATION) {
+            if (threadRefCount.decrementAndGet() == 0 && index.getFlushStatus(index)) {
+                ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
+                        NoOpOperationCallback.INSTANCE);
+                accessor.scheduleFlush(NoOpIOOperationCallback.INSTANCE);
+            }
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/TreeIndexFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/TreeIndexFactory.java
new file mode 100644
index 0000000..f570058
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/TreeIndexFactory.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManagerFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public abstract class TreeIndexFactory<T extends ITreeIndex> extends IndexFactory<T> {
+
+    protected final ITreeIndexFrameFactory interiorFrameFactory;
+    protected final ITreeIndexFrameFactory leafFrameFactory;
+    protected final IBinaryComparatorFactory[] cmpFactories;
+    protected final int fieldCount;
+
+    public TreeIndexFactory(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            IFreePageManagerFactory freePageManagerFactory, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount) {
+        super(bufferCache, fileMapProvider, freePageManagerFactory);
+        this.interiorFrameFactory = interiorFrameFactory;
+        this.leafFrameFactory = leafFrameFactory;
+        this.cmpFactories = cmpFactories;
+        this.fieldCount = fieldCount;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml b/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
new file mode 100644
index 0000000..2e74e52
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
@@ -0,0 +1,41 @@
+<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>hyracks-storage-am-lsm-invertedindex</artifactId>
+
+	<parent>
+		<artifactId>hyracks</artifactId>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<version>0.2.3-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>2.0.2</version>
+				<configuration>
+					<source>1.7</source>
+					<target>1.7</target>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+	<dependencies>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-btree</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-common</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+	    </dependency>
+	</dependencies>
+</project>
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
new file mode 100644
index 0000000..2556a25
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndex.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2012 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.api;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+public interface IInvertedIndex extends IIndex {
+    public IInvertedListCursor createInvertedListCursor();
+    
+    public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey, IIndexOperationContext ictx)
+            throws HyracksDataException, IndexException;
+    
+    public ITypeTraits[] getInvListTypeTraits();
+    
+    public IBinaryComparatorFactory[] getInvListCmpFactories();    
+    
+    public ITypeTraits[] getTokenTypeTraits();
+    
+    public IBinaryComparatorFactory[] getTokenCmpFactories();
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexAccessor.java
new file mode 100644
index 0000000..3fe5b57
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexAccessor.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 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.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+public interface IInvertedIndexAccessor extends IIndexAccessor {
+    public IInvertedListCursor createInvertedListCursor();
+
+    public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey)
+            throws HyracksDataException, IndexException;
+
+    public IIndexCursor createRangeSearchCursor();
+
+    public void rangeSearch(IIndexCursor cursor, ISearchPredicate searchPred) throws IndexException,
+            HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexFileNameMapper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexFileNameMapper.java
new file mode 100644
index 0000000..d7ec129
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexFileNameMapper.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 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.api;
+
+/**
+ * Maps from the dictionary BTree file/path to a corresponding inverted-lists file/path.
+ */
+public interface IInvertedIndexFileNameMapper {
+    public String getInvListsFilePath(String dictBTreeFilePath);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexOperatorDescriptor.java
new file mode 100644
index 0000000..60d1b03
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexOperatorDescriptor.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 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.api;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+
+public interface IInvertedIndexOperatorDescriptor extends IIndexOperatorDescriptor {
+    
+    public ITypeTraits[] getInvListsTypeTraits();
+    
+    public IBinaryComparatorFactory[] getInvListsComparatorFactories();
+    
+    public ITypeTraits[] getTokenTypeTraits();
+    
+    public IBinaryComparatorFactory[] getTokenComparatorFactories();
+
+    public IBinaryTokenizerFactory getTokenizerFactory();        
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifier.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifier.java
new file mode 100644
index 0000000..0d0d936
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifier.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2010 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.api;
+
+public interface IInvertedIndexSearchModifier {
+    public int getOccurrenceThreshold(int numQueryTokens);
+
+    public int getNumPrefixLists(int occurrenceThreshold, int numInvLists);
+
+    public short getNumTokensLowerBound(short numQueryTokens);
+
+    public short getNumTokensUpperBound(short numQueryTokens);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifierFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifierFactory.java
new file mode 100644
index 0000000..180f491
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifierFactory.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 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.api;
+
+import java.io.Serializable;
+
+public interface IInvertedIndexSearchModifierFactory extends Serializable {
+    public IInvertedIndexSearchModifier createSearchModifier();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearcher.java
new file mode 100644
index 0000000..0ea9383
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearcher.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2010 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.api;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexSearchCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
+
+public interface IInvertedIndexSearcher {
+    public void search(OnDiskInvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred, IIndexOperationContext ictx)
+            throws HyracksDataException, IndexException;
+
+    public IFrameTupleAccessor createResultFrameTupleAccessor();
+
+    public ITupleReference createResultFrameTupleReference();
+
+    public List<ByteBuffer> getResultBuffers();
+
+    public int getNumValidResultBuffers();
+    
+    public void reset();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedListBuilder.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedListBuilder.java
new file mode 100644
index 0000000..dcb5fdd
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedListBuilder.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2012 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.api;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface IInvertedListBuilder {
+    public boolean startNewList(ITupleReference tuple, int numTokenFields);
+
+    // returns true if successfully appended
+    // returns false if not enough space in targetBuf
+    public boolean appendElement(ITupleReference tuple, int numTokenFields, int numElementFields);
+
+    public void setTargetBuffer(byte[] targetBuf, int startPos);
+
+    public int getListSize();
+
+    public int getPos();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedListBuilderFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedListBuilderFactory.java
new file mode 100644
index 0000000..9cde18b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedListBuilderFactory.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2009-2012 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.api;
+
+public interface IInvertedListBuilderFactory {
+    public IInvertedListBuilder create();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
new file mode 100644
index 0000000..de703ac
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2010 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.api;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+
+public interface IInvertedListCursor extends Comparable<IInvertedListCursor> {
+    public void reset(int startPageId, int endPageId, int startOff, int numElements);
+
+    public void pinPages() throws HyracksDataException, IndexException;
+
+    public void unpinPages() throws HyracksDataException;
+
+    public boolean hasNext() throws HyracksDataException, IndexException;
+
+    public void next() throws HyracksDataException;
+
+    public ITupleReference getTuple();
+
+    // getters
+    public int size();
+
+    public int getStartPageId();
+
+    public int getEndPageId();
+
+    public int getStartOff();
+
+    public boolean containsKey(ITupleReference searchTuple, MultiComparator invListCmp) throws HyracksDataException, IndexException;
+    
+    // for debugging
+    @SuppressWarnings("rawtypes")
+    public String printInvList(ISerializerDeserializer[] serdes) throws HyracksDataException, IndexException;
+
+    @SuppressWarnings("rawtypes")
+    public String printCurrentElement(ISerializerDeserializer[] serdes) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IObjectFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IObjectFactory.java
new file mode 100644
index 0000000..9068a2b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IObjectFactory.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2009-2012 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.api;
+
+public interface IObjectFactory<T> {
+    public T create();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IPartitionedInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IPartitionedInvertedIndex.java
new file mode 100644
index 0000000..89fd69d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IPartitionedInvertedIndex.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2012 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.api;
+
+import java.util.ArrayList;
+
+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.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedListPartitions;
+
+public interface IPartitionedInvertedIndex {
+    public boolean openInvertedListPartitionCursors(IInvertedIndexSearcher searcher, IIndexOperationContext ictx,
+            short numTokensLowerBound, short numTokensUpperBound, InvertedListPartitions invListPartitions,
+            ArrayList<IInvertedListCursor> cursorsOrderedByTokens) throws HyracksDataException, IndexException;
+
+    public boolean isEmpty();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
new file mode 100644
index 0000000..e0af36e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/AbstractLSMInvertedIndexOperatorDescriptor.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2012 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.dataflow;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+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.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+
+public abstract class AbstractLSMInvertedIndexOperatorDescriptor extends AbstractIndexOperatorDescriptor implements
+        IInvertedIndexOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final ITypeTraits[] invListsTypeTraits;
+    protected final IBinaryComparatorFactory[] invListComparatorFactories;
+    protected final ITypeTraits[] tokenTypeTraits;
+    protected final IBinaryComparatorFactory[] tokenComparatorFactories;
+    protected final IBinaryTokenizerFactory tokenizerFactory;
+
+    public AbstractLSMInvertedIndexOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity,
+            int outputArity, RecordDescriptor recDesc, IStorageManagerInterface storageManager,
+            IFileSplitProvider fileSplitProvider, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
+            ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
+            IBinaryTokenizerFactory tokenizerFactory, IIndexDataflowHelperFactory dataflowHelperFactory,
+            ITupleFilterFactory tupleFilterFactory, boolean retainInput,
+            ILocalResourceFactoryProvider localResourceFactoryProvider,
+            ISearchOperationCallbackFactory searchOpCallbackFactory,
+            IModificationOperationCallbackFactory modificationOpCallbackFactory) {
+        super(spec, inputArity, outputArity, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider,
+                dataflowHelperFactory, tupleFilterFactory, retainInput, localResourceFactoryProvider,
+                searchOpCallbackFactory, modificationOpCallbackFactory);
+        this.invListsTypeTraits = invListsTypeTraits;
+        this.invListComparatorFactories = invListComparatorFactories;
+        this.tokenTypeTraits = tokenTypeTraits;
+        this.tokenComparatorFactories = tokenComparatorFactories;
+        this.tokenizerFactory = tokenizerFactory;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getTokenComparatorFactories() {
+        return tokenComparatorFactories;
+    }
+
+    @Override
+    public ITypeTraits[] getTokenTypeTraits() {
+        return tokenTypeTraits;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getInvListsComparatorFactories() {
+        return invListComparatorFactories;
+    }
+
+    @Override
+    public IBinaryTokenizerFactory getTokenizerFactory() {
+        return tokenizerFactory;
+    }
+
+    @Override
+    public ITypeTraits[] getInvListsTypeTraits() {
+        return invListsTypeTraits;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
new file mode 100644
index 0000000..84152d5
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2010 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+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;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+
+public class BinaryTokenizerOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private final IBinaryTokenizerFactory tokenizerFactory;
+    // Field that will be tokenized.
+    private final int docField;
+    // operator will append these key fields to each token, e.g., as
+    // payload for an inverted list
+    // WARNING: too many key fields can cause significant data blowup.
+    private final int[] keyFields;
+    // Indicates whether the first key field should be the number of tokens in the tokenized set of the document.
+    // This value is used in partitioned inverted indexes, for example.
+    private final boolean addNumTokensKey;
+
+    public BinaryTokenizerOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
+            IBinaryTokenizerFactory tokenizerFactory, int docField, int[] keyFields, boolean addNumTokensKey) {
+        super(spec, 1, 1);
+        this.tokenizerFactory = tokenizerFactory;
+        this.docField = docField;
+        this.keyFields = keyFields;
+        this.addNumTokensKey = addNumTokensKey;
+        recordDescriptors[0] = recDesc;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        return new BinaryTokenizerOperatorNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(
+                getActivityId(), 0), recordDescriptors[0], tokenizerFactory.createTokenizer(), docField, keyFields,
+                addNumTokensKey);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
new file mode 100644
index 0000000..6f28f61
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2009-2012 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.dataflow;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+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.data.std.util.GrowableArray;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+
+public class BinaryTokenizerOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+    private final IHyracksTaskContext ctx;
+    private final IBinaryTokenizer tokenizer;
+    private final int docField;
+    private final int[] keyFields;
+    private final boolean addNumTokensKey;
+    private final RecordDescriptor inputRecDesc;
+    private final RecordDescriptor outputRecDesc;
+
+    private FrameTupleAccessor accessor;
+    private ArrayTupleBuilder builder;
+    private GrowableArray builderData;
+    private FrameTupleAppender appender;
+    private ByteBuffer writeBuffer;
+
+    public BinaryTokenizerOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc,
+            RecordDescriptor outputRecDesc, IBinaryTokenizer tokenizer, int docField, int[] keyFields,
+            boolean addNumTokensKey) {
+        this.ctx = ctx;
+        this.tokenizer = tokenizer;
+        this.docField = docField;
+        this.keyFields = keyFields;
+        this.addNumTokensKey = addNumTokensKey;
+        this.inputRecDesc = inputRecDesc;
+        this.outputRecDesc = outputRecDesc;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+        writeBuffer = ctx.allocateFrame();
+        builder = new ArrayTupleBuilder(outputRecDesc.getFieldCount());
+        builderData = builder.getFieldData();
+        appender = new FrameTupleAppender(ctx.getFrameSize());
+        appender.reset(writeBuffer, true);
+        writer.open();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            short numTokens = 0;
+            if (addNumTokensKey) {
+                // Run through the tokens to get the total number of tokens.
+                tokenizer.reset(
+                        accessor.getBuffer().array(),
+                        accessor.getTupleStartOffset(i) + accessor.getFieldSlotsLength()
+                                + accessor.getFieldStartOffset(i, docField), accessor.getFieldLength(i, docField));
+                while (tokenizer.hasNext()) {
+                    tokenizer.next();
+                    numTokens++;
+                }
+            }
+
+            tokenizer.reset(
+                    accessor.getBuffer().array(),
+                    accessor.getTupleStartOffset(i) + accessor.getFieldSlotsLength()
+                            + accessor.getFieldStartOffset(i, docField), accessor.getFieldLength(i, docField));
+            while (tokenizer.hasNext()) {
+                tokenizer.next();
+
+                builder.reset();
+                try {
+                    IToken token = tokenizer.getToken();
+                    token.serializeToken(builderData);
+                    builder.addFieldEndOffset();
+                    // Add number of tokens if requested.
+                    if (addNumTokensKey) {
+                        builder.getDataOutput().writeShort(numTokens);
+                        builder.addFieldEndOffset();
+                    }
+                } catch (IOException e) {
+                    throw new HyracksDataException(e.getMessage());
+                }
+
+                for (int k = 0; k < keyFields.length; k++) {
+                    builder.addField(accessor, i, keyFields[k]);
+                }
+
+                if (!appender.append(builder.getFieldEndOffsets(), builder.getByteArray(), 0, builder.getSize())) {
+                    FrameUtils.flushFrame(writeBuffer, writer);
+                    appender.reset(writeBuffer, true);
+                    if (!appender.append(builder.getFieldEndOffsets(), builder.getByteArray(), 0, builder.getSize())) {
+                        throw new IllegalStateException();
+                    }
+                }
+            }
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (appender.getTupleCount() > 0) {
+            FrameUtils.flushFrame(writeBuffer, writer);
+        }
+        writer.close();
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        writer.fail();
+    }
+}
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
new file mode 100644
index 0000000..da3cad5
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexBulkLoadOperatorDescriptor.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+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.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexBulkLoadOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
+
+public class LSMInvertedIndexBulkLoadOperatorDescriptor extends AbstractLSMInvertedIndexOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int[] fieldPermutation;
+    private final boolean verifyInput;
+    private final long numElementsHint;
+
+    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,
+            IModificationOperationCallbackFactory modificationOpCallbackFactory) {
+        super(spec, 1, 0, null, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
+                tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
+                invertedIndexDataflowHelperFactory, null, false, NoOpLocalResourceFactoryProvider.INSTANCE,
+                NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
+        this.fieldPermutation = fieldPermutation;
+        this.verifyInput = verifyInput;
+        this.numElementsHint = numElementsHint;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new IndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, 1.0f, verifyInput,
+                numElementsHint, recordDescProvider);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java
new file mode 100644
index 0000000..641878f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexCreateOperatorDescriptor.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2010 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+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.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexCreateOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+
+public class LSMInvertedIndexCreateOperatorDescriptor extends AbstractLSMInvertedIndexOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public LSMInvertedIndexCreateOperatorDescriptor(IOperatorDescriptorRegistry spec,
+            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
+            IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
+            IIndexDataflowHelperFactory btreeDataflowHelperFactory,
+            ILocalResourceFactoryProvider localResourceFactoryProvider,
+            IModificationOperationCallbackFactory modificationOpCallbackFactory) {
+        super(spec, 0, 0, null, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
+                tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
+                btreeDataflowHelperFactory, null, false, localResourceFactoryProvider,
+                NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new IndexCreateOperatorNodePushable(this, ctx, partition);
+    }
+}
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
new file mode 100644
index 0000000..3d8b391
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2012 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public final class LSMInvertedIndexDataflowHelper extends AbstractLSMIndexDataflowHelper {
+
+    public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, mergePolicy, opTrackerFactory,
+                ioScheduler, ioOpCallbackProvider);
+    }
+
+    public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
+    }
+
+    @Override
+    public IIndex createIndexInstance() throws HyracksDataException {
+        IInvertedIndexOperatorDescriptor invIndexOpDesc = (IInvertedIndexOperatorDescriptor) opDesc;
+        try {
+            ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+            IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(),
+                    memPageSize, memNumPages);
+            IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
+                    metaDataFrameFactory);
+            IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
+            IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
+            LSMInvertedIndex invIndex = InvertedIndexUtils.createLSMInvertedIndex(memBufferCache, memFreePageManager,
+                    diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
+                    invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
+                    invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
+                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), mergePolicy, opTrackerFactory,
+                    ioScheduler, ioOpCallbackProvider, partition);
+            return invIndex;
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
new file mode 100644
index 0000000..9796ebc
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2009-2012 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
+
+public class LSMInvertedIndexDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public LSMInvertedIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
+            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+        super(mergePolicyProvider, opTrackerProvider, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
+                memNumPages);
+    }
+
+    @Override
+    public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition) {
+        return new LSMInvertedIndexDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicyProvider.getMergePolicy(ctx),
+                opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider);
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
new file mode 100644
index 0000000..963c653
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexInsertUpdateDeleteOperator.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2012 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+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.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+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.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
+
+public class LSMInvertedIndexInsertUpdateDeleteOperator extends AbstractLSMInvertedIndexOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int[] fieldPermutation;
+    private final IndexOperation op;
+
+    public LSMInvertedIndexInsertUpdateDeleteOperator(IOperatorDescriptorRegistry spec,
+            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
+            IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
+            int[] fieldPermutation, IndexOperation op, IIndexDataflowHelperFactory dataflowHelperFactory,
+            IModificationOperationCallbackFactory modificationOpCallbackFactory) {
+        super(spec, 1, 1, null, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
+                tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, tokenizerFactory,
+                dataflowHelperFactory, null, false, NoOpLocalResourceFactoryProvider.INSTANCE,
+                NoOpOperationCallbackFactory.INSTANCE, modificationOpCallbackFactory);
+        this.fieldPermutation = fieldPermutation;
+        this.op = op;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new LSMIndexInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
+                recordDescProvider, op);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
new file mode 100644
index 0000000..4f97060
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2012 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+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.IOperatorDescriptorRegistry;
+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.ISearchOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
+
+public class LSMInvertedIndexSearchOperatorDescriptor extends AbstractLSMInvertedIndexOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final int queryField;
+    private final IInvertedIndexSearchModifierFactory searchModifierFactory;
+
+    public LSMInvertedIndexSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, int queryField,
+            IStorageManagerInterface storageManager, IFileSplitProvider fileSplitProvider,
+            IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
+            IBinaryComparatorFactory[] invListComparatorFactories,
+            IIndexDataflowHelperFactory btreeDataflowHelperFactory, IBinaryTokenizerFactory queryTokenizerFactory,
+            IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc, boolean retainInput,
+            ISearchOperationCallbackFactory searchOpCallbackProvider) {
+        super(spec, 1, 1, recDesc, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
+                tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, queryTokenizerFactory,
+                btreeDataflowHelperFactory, null, retainInput, NoOpLocalResourceFactoryProvider.INSTANCE,
+                searchOpCallbackProvider, NoOpOperationCallbackFactory.INSTANCE);
+        this.queryField = queryField;
+        this.searchModifierFactory = searchModifierFactory;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        IInvertedIndexSearchModifier searchModifier = searchModifierFactory.createSearchModifier();
+        return new LSMInvertedIndexSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, queryField,
+                searchModifier);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
new file mode 100644
index 0000000..d825c02
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2012 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
+
+public class LSMInvertedIndexSearchOperatorNodePushable extends IndexSearchOperatorNodePushable {
+
+    protected final IInvertedIndexSearchModifier searchModifier;
+    protected final int queryFieldIndex;
+    protected final int invListFields;
+
+    public LSMInvertedIndexSearchOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, IRecordDescriptorProvider recordDescProvider, int queryFieldIndex,
+            IInvertedIndexSearchModifier searchModifier) {
+        super(opDesc, ctx, partition, recordDescProvider);
+        this.searchModifier = searchModifier;
+        this.queryFieldIndex = queryFieldIndex;
+        // If retainInput is true, the frameTuple is created in IndexSearchOperatorNodePushable.open().
+        if (!opDesc.getRetainInput()) {
+            this.frameTuple = new FrameTupleReference();
+        }
+        AbstractLSMInvertedIndexOperatorDescriptor invIndexOpDesc = (AbstractLSMInvertedIndexOperatorDescriptor) opDesc;
+        invListFields = invIndexOpDesc.getInvListsTypeTraits().length;
+    }
+
+    @Override
+    protected ISearchPredicate createSearchPredicate() {
+        AbstractLSMInvertedIndexOperatorDescriptor invIndexOpDesc = (AbstractLSMInvertedIndexOperatorDescriptor) opDesc;
+        return new InvertedIndexSearchPredicate(invIndexOpDesc.getTokenizerFactory().createTokenizer(), searchModifier);
+    }
+
+    @Override
+    protected void resetSearchPredicate(int tupleIndex) {
+        frameTuple.reset(accessor, tupleIndex);
+        InvertedIndexSearchPredicate invIndexSearchPred = (InvertedIndexSearchPredicate) searchPred;
+        invIndexSearchPred.setQueryTuple(frameTuple);
+        invIndexSearchPred.setQueryFieldIndex(queryFieldIndex);
+    }
+}
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
new file mode 100644
index 0000000..c5b4f07
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2012 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.PartitionedLSMInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public final class PartitionedLSMInvertedIndexDataflowHelper extends AbstractLSMIndexDataflowHelper {
+
+    public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, mergePolicy, opTrackerFactory,
+                ioScheduler, ioOpCallbackProvider);
+    }
+
+    public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
+    }
+
+    @Override
+    public IIndex createIndexInstance() throws HyracksDataException {
+        IInvertedIndexOperatorDescriptor invIndexOpDesc = (IInvertedIndexOperatorDescriptor) opDesc;
+        try {
+            ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+            IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(),
+                    memPageSize, memNumPages);
+            IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
+                    metaDataFrameFactory);
+            IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
+            IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
+            PartitionedLSMInvertedIndex invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(memBufferCache,
+                    memFreePageManager, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
+                    invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
+                    invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
+                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), mergePolicy, opTrackerFactory,
+                    ioScheduler, ioOpCallbackProvider, partition);
+            return invIndex;
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
new file mode 100644
index 0000000..8a8aad2
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2012 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
+
+public class PartitionedLSMInvertedIndexDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public PartitionedLSMInvertedIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
+            ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+        super(mergePolicyProvider, opTrackerProvider, ioSchedulerProvider, ioOpCallbackProvider, memNumPages,
+                memNumPages);
+    }
+
+    @Override
+    public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition) {
+        return new PartitionedLSMInvertedIndexDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
+                mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
+                ioOpCallbackProvider);
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/exceptions/InvertedIndexException.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/exceptions/InvertedIndexException.java
new file mode 100644
index 0000000..cc7ff87
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/exceptions/InvertedIndexException.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 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.exceptions;
+
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+
+public class InvertedIndexException extends IndexException {
+    private static final long serialVersionUID = 1L;
+
+    public InvertedIndexException(Exception e) {        
+        super(e);
+    }
+    
+    public InvertedIndexException(String msg) {
+        super(msg);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/exceptions/OccurrenceThresholdPanicException.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/exceptions/OccurrenceThresholdPanicException.java
new file mode 100644
index 0000000..bedaa60
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/exceptions/OccurrenceThresholdPanicException.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2010 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.exceptions;
+
+
+
+public class OccurrenceThresholdPanicException extends InvertedIndexException {
+    private static final long serialVersionUID = 1L;
+
+    public OccurrenceThresholdPanicException(String msg) {
+        super(msg);
+    }
+}
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
new file mode 100644
index 0000000..c69a8df
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -0,0 +1,732 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+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.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+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.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex {
+
+    // In-memory components.
+    protected final LSMInvertedIndexMutableComponent mutableComponent;
+    protected final IInMemoryFreePageManager memFreePageManager;
+    protected final IBinaryTokenizerFactory tokenizerFactory;
+
+    // On-disk components.
+    // For creating inverted indexes in flush and merge.
+    protected final ILSMComponentFactory componentFactory;
+
+    // Type traits and comparators for tokens and inverted-list elements.
+    protected final ITypeTraits[] invListTypeTraits;
+    protected final IBinaryComparatorFactory[] invListCmpFactories;
+    protected final ITypeTraits[] tokenTypeTraits;
+    protected final IBinaryComparatorFactory[] tokenCmpFactories;
+
+    public LSMInvertedIndex(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
+            OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
+            BloomFilterFactory bloomFilterFactory, ILSMIndexFileManager fileManager,
+            IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+            throws IndexException {
+        super(memFreePageManager, diskInvIndexFactory.getBufferCache(), fileManager, diskFileMapProvider, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        this.memFreePageManager = memFreePageManager;
+        this.tokenizerFactory = tokenizerFactory;
+        this.invListTypeTraits = invListTypeTraits;
+        this.invListCmpFactories = invListCmpFactories;
+        this.tokenTypeTraits = tokenTypeTraits;
+        this.tokenCmpFactories = tokenCmpFactories;
+        // Create in-memory component.
+        InMemoryInvertedIndex memInvIndex = createInMemoryInvertedIndex(memBufferCache);
+        BTree deleteKeysBTree = BTreeUtils.createBTree(memBufferCache, memFreePageManager,
+                ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), invListTypeTraits, invListCmpFactories,
+                BTreeLeafFrameType.REGULAR_NSM, new FileReference(new File("membtree")));
+        mutableComponent = new LSMInvertedIndexMutableComponent(memInvIndex, deleteKeysBTree, memFreePageManager);
+        componentFactory = new LSMInvertedIndexComponentFactory(diskInvIndexFactory, deletedKeysBTreeFactory,
+                bloomFilterFactory);
+    }
+
+    @Override
+    public synchronized void create() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to create the index since it is activated.");
+        }
+
+        fileManager.deleteDirs();
+        fileManager.createDirs();
+        componentsRef.get().clear();
+    }
+
+    @Override
+    public synchronized void activate() throws HyracksDataException {
+        if (isActivated) {
+            return;
+        }
+        try {
+            List<ILSMComponent> immutableComponents = componentsRef.get();
+            ((InMemoryBufferCache) mutableComponent.getInvIndex().getBufferCache()).open();
+            mutableComponent.getInvIndex().create();
+            mutableComponent.getInvIndex().activate();
+            mutableComponent.getDeletedKeysBTree().create();
+            mutableComponent.getDeletedKeysBTree().activate();
+            immutableComponents.clear();
+            List<LSMComponentFileReferences> validFileReferences = fileManager.cleanupAndGetValidFiles();
+            for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
+                LSMInvertedIndexImmutableComponent component;
+                try {
+                    component = createDiskInvIndexComponent(componentFactory,
+                            lsmComonentFileReference.getInsertIndexFileReference(),
+                            lsmComonentFileReference.getDeleteIndexFileReference(),
+                            lsmComonentFileReference.getBloomFilterFileReference(), false);
+                } catch (IndexException e) {
+                    throw new HyracksDataException(e);
+                }
+                immutableComponents.add(component);
+            }
+            isActivated = true;
+            // TODO: Maybe we can make activate throw an index exception?
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void clear() throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("Failed to clear the index since it is not activated.");
+        }
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        mutableComponent.getInvIndex().clear();
+        mutableComponent.getDeletedKeysBTree().clear();
+        for (ILSMComponent c : immutableComponents) {
+            LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
+            component.getBloomFilter().deactivate();
+            component.getInvIndex().deactivate();
+            component.getDeletedKeysBTree().deactivate();
+            component.getBloomFilter().destroy();
+            component.getInvIndex().destroy();
+            component.getDeletedKeysBTree().destroy();
+        }
+        immutableComponents.clear();
+    }
+
+    @Override
+    public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
+        if (!isActivated) {
+            return;
+        }
+
+        isActivated = false;
+
+        if (flushOnExit) {
+            BlockingIOOperationCallbackWrapper blockingCallBack = new BlockingIOOperationCallbackWrapper(
+                    ioOpCallbackProvider.getIOOperationCallback(this));
+            ILSMIndexAccessor accessor = (ILSMIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            accessor.scheduleFlush(blockingCallBack);
+            try {
+                blockingCallBack.waitForIO();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
+            component.getBloomFilter().deactivate();
+            component.getInvIndex().deactivate();
+            component.getDeletedKeysBTree().deactivate();
+        }
+        mutableComponent.getInvIndex().deactivate();
+        mutableComponent.getDeletedKeysBTree().deactivate();
+        mutableComponent.getInvIndex().destroy();
+        mutableComponent.getDeletedKeysBTree().destroy();
+        ((InMemoryBufferCache) mutableComponent.getInvIndex().getBufferCache()).close();
+    }
+
+    @Override
+    public synchronized void deactivate() throws HyracksDataException {
+        deactivate(true);
+    }
+
+    @Override
+    public synchronized void destroy() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to destroy the index since it is activated.");
+        }
+
+        mutableComponent.getInvIndex().destroy();
+        mutableComponent.getDeletedKeysBTree().destroy();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
+            component.getInvIndex().destroy();
+            component.getDeletedKeysBTree().destroy();
+            component.getBloomFilter().destroy();
+        }
+        fileManager.deleteDirs();
+    }
+
+    @Override
+    public void getOperationalComponents(ILSMIndexOperationContext ctx) {
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
+        operationalComponents.clear();
+        switch (ctx.getOperation()) {
+            case FLUSH:
+            case DELETE:
+            case INSERT:
+                operationalComponents.add(mutableComponent);
+                break;
+            case SEARCH:
+                operationalComponents.add(mutableComponent);
+                operationalComponents.addAll(immutableComponents);
+                break;
+            case MERGE:
+                operationalComponents.addAll(immutableComponents);
+                break;
+            default:
+                throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
+        }
+    }
+
+    /**
+     * The keys in the in-memory deleted-keys BTree only refer to on-disk components.
+     * We delete documents from the in-memory inverted index by deleting its entries directly,
+     * while still adding the deleted key to the deleted-keys BTree.
+     * Otherwise, inserts would have to remove keys from the in-memory deleted-keys BTree which
+     * may cause incorrect behavior (lost deletes) in the following pathological case:
+     * Insert doc 1, flush, delete doc 1, insert doc 1
+     * After the sequence above doc 1 will now appear twice because the delete of the on-disk doc 1 has been lost.
+     * Insert:
+     * - Insert document into in-memory inverted index.
+     * Delete:
+     * - Delete document from in-memory inverted index (ignore if it does not exist).
+     * - Insert key into deleted-keys BTree.
+     */
+    @Override
+    public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException {
+        LSMInvertedIndexOpContext ctx = (LSMInvertedIndexOpContext) ictx;
+        // TODO: This is a hack to support logging properly in ASTERIX.
+        // The proper undo operations are only dependent on the after image so 
+        // it is correct to say we found nothing (null) as the before image (at least 
+        // in the perspective of ASTERIX). The semantics for the operation callbacks 
+        // are violated here (and they are somewhat unclear in the first place as to 
+        // what they should be for an inverted index).
+        ctx.modificationCallback.before(tuple);
+        ctx.modificationCallback.found(null, tuple);
+        switch (ctx.getOperation()) {
+            case INSERT: {
+                // Insert into the in-memory inverted index.                
+                ctx.memInvIndexAccessor.insert(tuple);
+                break;
+            }
+            case DELETE: {
+                // First remove all entries in the in-memory inverted index (if any).
+                ctx.memInvIndexAccessor.delete(tuple);
+                // Insert key into the deleted-keys BTree.
+                ctx.keysOnlyTuple.reset(tuple);
+                try {
+                    ctx.deletedKeysBTreeAccessor.insert(ctx.keysOnlyTuple);
+                } catch (BTreeDuplicateKeyException e) {
+                    // Key has already been deleted.
+                }
+                break;
+            }
+            default: {
+                throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
+            }
+        }
+    }
+
+    @Override
+    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException {
+        List<ILSMComponent> operationalComponents = ictx.getComponentHolder();
+        int numComponents = operationalComponents.size();
+        assert numComponents > 0;
+        boolean includeMutableComponent = operationalComponents.get(0) == mutableComponent;
+        ArrayList<IIndexAccessor> indexAccessors = new ArrayList<IIndexAccessor>(numComponents);
+        ArrayList<IIndexAccessor> deletedKeysBTreeAccessors = new ArrayList<IIndexAccessor>(numComponents);
+        if (includeMutableComponent) {
+            IIndexAccessor invIndexAccessor = mutableComponent.getInvIndex().createAccessor(
+                    NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            indexAccessors.add(invIndexAccessor);
+            IIndexAccessor deletedKeysAccessor = mutableComponent.getDeletedKeysBTree().createAccessor(
+                    NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            deletedKeysBTreeAccessors.add(deletedKeysAccessor);
+        }
+
+        for (int i = includeMutableComponent ? 1 : 0; i < operationalComponents.size(); i++) {
+            LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) operationalComponents
+                    .get(i);
+            IIndexAccessor invIndexAccessor = component.getInvIndex().createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            indexAccessors.add(invIndexAccessor);
+            IIndexAccessor deletedKeysAccessor = component.getDeletedKeysBTree().createAccessor(
+                    NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            deletedKeysBTreeAccessors.add(deletedKeysAccessor);
+        }
+
+        ICursorInitialState initState = createCursorInitialState(pred, ictx, includeMutableComponent, indexAccessors,
+                deletedKeysBTreeAccessors);
+        cursor.open(initState, pred);
+    }
+
+    private ICursorInitialState createCursorInitialState(ISearchPredicate pred, IIndexOperationContext ictx,
+            boolean includeMutableComponent, ArrayList<IIndexAccessor> indexAccessors,
+            ArrayList<IIndexAccessor> deletedKeysBTreeAccessors) {
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        ICursorInitialState initState = null;
+        PermutingTupleReference keysOnlyTuple = createKeysOnlyTupleReference();
+        MultiComparator keyCmp = MultiComparator.createIgnoreFieldLength(invListCmpFactories);
+        List<ILSMComponent> operationalComponents = new ArrayList<ILSMComponent>();
+        if (includeMutableComponent) {
+            operationalComponents.add(mutableComponent);
+        }
+        operationalComponents.addAll(immutableComponents);
+
+        // TODO: This check is not pretty, but it does the job. Come up with something more OO in the future.
+        // Distinguish between regular searches and range searches (mostly used in merges).
+        if (pred instanceof InvertedIndexSearchPredicate) {
+            initState = new LSMInvertedIndexSearchCursorInitialState(keyCmp, keysOnlyTuple, indexAccessors,
+                    deletedKeysBTreeAccessors, mutableComponent.getDeletedKeysBTree().getLeafFrameFactory(), ictx,
+                    includeMutableComponent, lsmHarness, operationalComponents);
+        } else {
+            InMemoryInvertedIndex memInvIndex = (InMemoryInvertedIndex) mutableComponent.getInvIndex();
+            MultiComparator tokensAndKeysCmp = MultiComparator.create(memInvIndex.getBTree().getComparatorFactories());
+            initState = new LSMInvertedIndexRangeSearchCursorInitialState(tokensAndKeysCmp, keyCmp, keysOnlyTuple,
+                    mutableComponent.getDeletedKeysBTree().getLeafFrameFactory(), includeMutableComponent, lsmHarness,
+                    indexAccessors, deletedKeysBTreeAccessors, pred, operationalComponents);
+        }
+        return initState;
+    }
+
+    /**
+     * Returns a permuting tuple reference that projects away the document field(s) of a tuple, only leaving the key fields.
+     */
+    private PermutingTupleReference createKeysOnlyTupleReference() {
+        // Project away token fields.
+        int[] keyFieldPermutation = new int[invListTypeTraits.length];
+        int numTokenFields = tokenTypeTraits.length;
+        for (int i = 0; i < invListTypeTraits.length; i++) {
+            keyFieldPermutation[i] = numTokenFields + i;
+        }
+        return new PermutingTupleReference(keyFieldPermutation);
+    }
+
+    @Override
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException {
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        LSMInvertedIndexOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+        ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
+        opCtx.setOperation(IndexOperation.FLUSH);
+        opCtx.getComponentHolder().add(flushingComponent);
+        ioScheduler.scheduleOperation(new LSMInvertedIndexFlushOperation(new LSMInvertedIndexAccessor(this, lsmHarness,
+                fileManager, opCtx), mutableComponent, componentFileRefs.getInsertIndexFileReference(),
+                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
+                callback));
+    }
+
+    @Override
+    public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        LSMInvertedIndexFlushOperation flushOp = (LSMInvertedIndexFlushOperation) operation;
+
+        // Create an inverted index instance to be bulk loaded.
+        LSMInvertedIndexImmutableComponent component = createDiskInvIndexComponent(componentFactory,
+                flushOp.getDictBTreeFlushTarget(), flushOp.getDeletedKeysBTreeFlushTarget(),
+                flushOp.getBloomFilterFlushTarget(), true);
+        IInvertedIndex diskInvertedIndex = component.getInvIndex();
+
+        // Create a scan cursor on the BTree underlying the in-memory inverted index.
+        LSMInvertedIndexMutableComponent flushingComponent = flushOp.getFlushingComponent();
+        InMemoryInvertedIndexAccessor memInvIndexAccessor = (InMemoryInvertedIndexAccessor) flushingComponent
+                .getInvIndex().createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        BTreeAccessor memBTreeAccessor = memInvIndexAccessor.getBTreeAccessor();
+        RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
+        IIndexCursor scanCursor = memBTreeAccessor.createSearchCursor();
+        memBTreeAccessor.search(scanCursor, nullPred);
+
+        // Bulk load the disk inverted index from the in-memory inverted index.
+        IIndexBulkLoader invIndexBulkLoader = diskInvertedIndex.createBulkLoader(1.0f, false, 0L);
+        try {
+            while (scanCursor.hasNext()) {
+                scanCursor.next();
+                invIndexBulkLoader.add(scanCursor.getTuple());
+            }
+        } finally {
+            scanCursor.close();
+        }
+        invIndexBulkLoader.end();
+
+        IIndexAccessor deletedKeysBTreeAccessor = flushingComponent.getDeletedKeysBTree().createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        IIndexCursor btreeCountingCursor = ((BTreeAccessor) deletedKeysBTreeAccessor).createCountingSearchCursor();
+        deletedKeysBTreeAccessor.search(btreeCountingCursor, nullPred);
+        long numBTreeTuples = 0L;
+        try {
+            while (btreeCountingCursor.hasNext()) {
+                btreeCountingCursor.next();
+                ITupleReference countTuple = btreeCountingCursor.getTuple();
+                numBTreeTuples = IntegerSerializerDeserializer.getInt(countTuple.getFieldData(0),
+                        countTuple.getFieldStart(0));
+            }
+        } finally {
+            btreeCountingCursor.close();
+        }
+
+        if (numBTreeTuples > 0) {
+            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numBTreeTuples);
+            BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                    MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+
+            // Create an BTree instance for the deleted keys.
+            BTree diskDeletedKeysBTree = component.getDeletedKeysBTree();
+
+            // Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index.
+            IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor();
+            deletedKeysBTreeAccessor.search(deletedKeysScanCursor, nullPred);
+
+            // Bulk load the deleted-keys BTree.
+            IIndexBulkLoader deletedKeysBTreeBulkLoader = diskDeletedKeysBTree.createBulkLoader(1.0f, false, 0L);
+            IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numBTreeTuples,
+                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+
+            try {
+                while (deletedKeysScanCursor.hasNext()) {
+                    deletedKeysScanCursor.next();
+                    deletedKeysBTreeBulkLoader.add(deletedKeysScanCursor.getTuple());
+                    builder.add(deletedKeysScanCursor.getTuple());
+                }
+            } finally {
+                deletedKeysScanCursor.close();
+                builder.end();
+            }
+            deletedKeysBTreeBulkLoader.end();
+        }
+
+        return component;
+    }
+
+    @Override
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        LSMInvertedIndexOpContext ictx = createOpContext(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+        ictx.getComponentHolder().addAll(mergingComponents);
+        IIndexCursor cursor = new LSMInvertedIndexRangeSearchCursor(ictx);
+        RangePredicate mergePred = new RangePredicate(null, null, true, true, null, null);
+
+        // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
+        search(ictx, cursor, mergePred);
+
+        ictx.setOperation(IndexOperation.MERGE);
+        LSMInvertedIndexImmutableComponent firstComponent = (LSMInvertedIndexImmutableComponent) mergingComponents
+                .get(0);
+        OnDiskInvertedIndex firstInvIndex = (OnDiskInvertedIndex) firstComponent.getInvIndex();
+        String firstFileName = firstInvIndex.getBTree().getFileReference().getFile().getName();
+
+        LSMInvertedIndexImmutableComponent lastComponent = (LSMInvertedIndexImmutableComponent) mergingComponents
+                .get(mergingComponents.size() - 1);
+        OnDiskInvertedIndex lastInvIndex = (OnDiskInvertedIndex) lastComponent.getInvIndex();
+        String lastFileName = lastInvIndex.getBTree().getFileReference().getFile().getName();
+
+        LSMComponentFileReferences relMergeFileRefs = fileManager.getRelMergeFileReference(firstFileName, lastFileName);
+        ILSMIndexAccessorInternal accessor = new LSMInvertedIndexAccessor(this, lsmHarness, fileManager, ictx);
+        ioScheduler.scheduleOperation(new LSMInvertedIndexMergeOperation(accessor, mergingComponents, cursor,
+                relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getDeleteIndexFileReference(),
+                relMergeFileRefs.getBloomFilterFileReference(), callback));
+    }
+
+    @Override
+    public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
+            throws HyracksDataException, IndexException {
+        LSMInvertedIndexMergeOperation mergeOp = (LSMInvertedIndexMergeOperation) operation;
+
+        // Create an inverted index instance.
+        LSMInvertedIndexImmutableComponent component = createDiskInvIndexComponent(componentFactory,
+                mergeOp.getDictBTreeMergeTarget(), mergeOp.getDeletedKeysBTreeMergeTarget(),
+                mergeOp.getBloomFilterMergeTarget(), true);
+
+        IInvertedIndex mergedDiskInvertedIndex = component.getInvIndex();
+        IIndexCursor cursor = mergeOp.getCursor();
+        IIndexBulkLoader invIndexBulkLoader = mergedDiskInvertedIndex.createBulkLoader(1.0f, true, 0L);
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+                ITupleReference tuple = cursor.getTuple();
+                invIndexBulkLoader.add(tuple);
+            }
+        } finally {
+            cursor.close();
+        }
+        invIndexBulkLoader.end();
+
+        // Add the merged components for cleanup.
+        mergedComponents.addAll(mergeOp.getMergingComponents());
+
+        return component;
+    }
+
+    private ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        return createDiskInvIndexComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
+                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+            throws IndexException {
+        return new LSMInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint);
+    }
+
+    public class LSMInvertedIndexBulkLoader implements IIndexBulkLoader {
+        private final ILSMComponent component;
+        private final IIndexBulkLoader invIndexBulkLoader;
+
+        public LSMInvertedIndexBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+                throws IndexException {
+            // 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);
+            }
+            invIndexBulkLoader = ((LSMInvertedIndexImmutableComponent) component).getInvIndex().createBulkLoader(
+                    fillFactor, verifyInput, numElementsHint);
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
+            try {
+                invIndexBulkLoader.add(tuple);
+            } catch (IndexException e) {
+                handleException();
+                throw e;
+            } catch (HyracksDataException e) {
+                handleException();
+                throw e;
+            } catch (RuntimeException e) {
+                handleException();
+                throw e;
+            }
+        }
+
+        protected void handleException() throws HyracksDataException {
+            ((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 {
+            invIndexBulkLoader.end();
+            lsmHarness.addBulkLoadedComponent(component);
+        }
+    }
+
+    protected InMemoryInvertedIndex createInMemoryInvertedIndex(IInMemoryBufferCache memBufferCache)
+            throws IndexException {
+        return InvertedIndexUtils.createInMemoryBTreeInvertedindex(memBufferCache, memFreePageManager,
+                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
+    }
+
+    protected LSMInvertedIndexImmutableComponent createDiskInvIndexComponent(ILSMComponentFactory factory,
+            FileReference dictBTreeFileRef, FileReference btreeFileRef, FileReference bloomFilterFileRef, boolean create)
+            throws HyracksDataException, IndexException {
+        LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) factory
+                .createLSMComponentInstance(new LSMComponentFileReferences(dictBTreeFileRef, btreeFileRef,
+                        bloomFilterFileRef));
+        if (create) {
+            component.getInvIndex().create();
+            component.getDeletedKeysBTree().create();
+            component.getBloomFilter().create();
+        }
+        // Will be closed during cleanup of merge().
+        component.getInvIndex().activate();
+        component.getDeletedKeysBTree().activate();
+        component.getBloomFilter().activate();
+        return component;
+    }
+
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new LSMInvertedIndexAccessor(this, lsmHarness, fileManager, createOpContext(modificationCallback,
+                searchCallback));
+    }
+
+    private LSMInvertedIndexOpContext createOpContext(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new LSMInvertedIndexOpContext(mutableComponent.getInvIndex(), mutableComponent.getDeletedKeysBTree(),
+                modificationCallback, searchCallback);
+    }
+
+    @Override
+    public IInvertedListCursor createInvertedListCursor() {
+        throw new UnsupportedOperationException("Cannot create inverted list cursor on lsm inverted index.");
+    }
+
+    @Override
+    public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey,
+            IIndexOperationContext ictx) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Cannot open inverted list cursor on lsm inverted index.");
+    }
+
+    @Override
+    public ITypeTraits[] getInvListTypeTraits() {
+        return invListTypeTraits;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getInvListCmpFactories() {
+        return invListCmpFactories;
+    }
+
+    @Override
+    public long getMemoryAllocationSize() {
+        InMemoryBufferCache memBufferCache = (InMemoryBufferCache) mutableComponent.getInvIndex().getBufferCache();
+        return memBufferCache.getNumPages() * memBufferCache.getPageSize();
+    }
+
+    @Override
+    public ITypeTraits[] getTokenTypeTraits() {
+        return tokenTypeTraits;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getTokenCmpFactories() {
+        return tokenCmpFactories;
+    }
+
+    public IBinaryTokenizerFactory getTokenizerFactory() {
+        return tokenizerFactory;
+    }
+
+    protected void forceFlushInvListsFileDirtyPages(OnDiskInvertedIndex invIndex) throws HyracksDataException {
+        int fileId = invIndex.getInvListsFileId();
+        IBufferCache bufferCache = invIndex.getBufferCache();
+        int startPageId = 0;
+        int maxPageId = invIndex.getInvListsMaxPageId();
+        forceFlushDirtyPages(bufferCache, fileId, startPageId, maxPageId);
+    }
+
+    @Override
+    public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException {
+        LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) lsmComponent;
+        OnDiskInvertedIndex invIndex = (OnDiskInvertedIndex) invIndexComponent.getInvIndex();
+        // Flush the bloom filter first.
+        int fileId = invIndexComponent.getBloomFilter().getFileId();
+        IBufferCache bufferCache = invIndex.getBufferCache();
+        int startPage = 0;
+        int maxPage = invIndexComponent.getBloomFilter().getNumPages();
+        forceFlushDirtyPages(bufferCache, fileId, startPage, maxPage);
+
+        ITreeIndex treeIndex = invIndex.getBTree();
+        // Flush inverted index second.
+        forceFlushDirtyPages(treeIndex);
+        forceFlushInvListsFileDirtyPages(invIndex);
+        // Flush deleted keys BTree.
+        forceFlushDirtyPages(invIndexComponent.getDeletedKeysBTree());
+        // We use the dictionary BTree for marking the inverted index as valid.
+        markAsValidInternal(treeIndex);
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        mutableComponent.getInvIndex().validate();
+        mutableComponent.getDeletedKeysBTree().validate();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) c;
+            component.getInvIndex().validate();
+            component.getDeletedKeysBTree().validate();
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
new file mode 100644
index 0000000..7f7d3cd
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2009-2010 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.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+
+public class LSMInvertedIndexAccessor implements ILSMIndexAccessorInternal, IInvertedIndexAccessor {
+
+    protected final ILSMHarness lsmHarness;
+    protected final ILSMIndexFileManager fileManager;
+    protected final ILSMIndexOperationContext ctx;
+    protected final LSMInvertedIndex invIndex;
+
+    public LSMInvertedIndexAccessor(LSMInvertedIndex invIndex, ILSMHarness lsmHarness,
+            ILSMIndexFileManager fileManager, ILSMIndexOperationContext ctx) {
+        this.lsmHarness = lsmHarness;
+        this.fileManager = fileManager;
+        this.ctx = ctx;
+        this.invIndex = invIndex;
+    }
+
+    @Override
+    public void insert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.INSERT);
+        lsmHarness.modify(ctx, false, tuple);
+    }
+
+    @Override
+    public void delete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.DELETE);
+        lsmHarness.modify(ctx, false, tuple);
+    }
+
+    @Override
+    public boolean tryInsert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.INSERT);
+        return lsmHarness.modify(ctx, true, tuple);
+    }
+
+    @Override
+    public boolean tryDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.DELETE);
+        return lsmHarness.modify(ctx, true, tuple);
+    }
+
+    public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.SEARCH);
+        lsmHarness.search(ctx, cursor, searchPred);
+    }
+
+    public IIndexCursor createSearchCursor() {
+        return new LSMInvertedIndexSearchCursor();
+    }
+
+    @Override
+    public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+        ctx.setOperation(IndexOperation.FLUSH);
+        lsmHarness.scheduleFlush(ctx, callback);
+    }
+
+    @Override
+    public void flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        lsmHarness.flush(ctx, operation);
+    }
+
+    @Override
+    public void scheduleMerge(ILSMIOOperationCallback callback) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.MERGE);
+        lsmHarness.scheduleMerge(ctx, callback);
+    }
+
+    @Override
+    public void merge(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        lsmHarness.merge(ctx, operation);
+    }
+
+    @Override
+    public IIndexCursor createRangeSearchCursor() {
+        return new LSMInvertedIndexRangeSearchCursor(ctx);
+    }
+
+    @Override
+    public void rangeSearch(IIndexCursor cursor, ISearchPredicate searchPred) throws IndexException,
+            HyracksDataException {
+        search(cursor, searchPred);
+    }
+
+    @Override
+    public void noOp() throws HyracksDataException {
+        lsmHarness.noOp(ctx);
+    }
+
+    @Override
+    public void forcePhysicalDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Physical delete not supported by lsm inverted index.");
+    }
+
+    @Override
+    public void forceInsert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.INSERT);
+        lsmHarness.forceModify(ctx, tuple);
+    }
+
+    @Override
+    public void forceDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        ctx.setOperation(IndexOperation.DELETE);
+        lsmHarness.forceModify(ctx, tuple);
+    }
+
+    @Override
+    public void physicalDelete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Physical delete not supported by lsm inverted index.");
+    }
+
+    @Override
+    public void update(ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Update not supported by lsm inverted index.");
+    }
+
+    @Override
+    public void upsert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Upsert not supported by lsm inverted index.");
+    }
+
+    @Override
+    public boolean tryUpdate(ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Update not supported by lsm inverted index.");
+    }
+
+    @Override
+    public boolean tryUpsert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Upsert not supported by lsm inverted index.");
+    }
+
+    @Override
+    public IInvertedListCursor createInvertedListCursor() {
+        throw new UnsupportedOperationException("Cannot create inverted list cursor on lsm inverted index.");
+    }
+
+    @Override
+    public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey)
+            throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Cannot open inverted list cursor on lsm inverted index.");
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexComponentFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexComponentFactory.java
new file mode 100644
index 0000000..1f4db63
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexComponentFactory.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public class LSMInvertedIndexComponentFactory implements ILSMComponentFactory {
+    private final OnDiskInvertedIndexFactory diskInvIndexFactory;
+    private final TreeIndexFactory<BTree> btreeFactory;
+    private final BloomFilterFactory bloomFilterFactory;
+
+    public LSMInvertedIndexComponentFactory(OnDiskInvertedIndexFactory diskInvIndexFactory,
+            TreeIndexFactory<BTree> btreeFactory, BloomFilterFactory bloomFilterFactory) {
+        this.diskInvIndexFactory = diskInvIndexFactory;
+        this.btreeFactory = btreeFactory;
+        this.bloomFilterFactory = bloomFilterFactory;
+    }
+
+    @Override
+    public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) throws IndexException,
+            HyracksDataException {
+        return new LSMInvertedIndexImmutableComponent(diskInvIndexFactory.createIndexInstance(cfr
+                .getInsertIndexFileReference()), btreeFactory.createIndexInstance(cfr.getDeleteIndexFileReference()),
+                bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()));
+    }
+
+    @Override
+    public IBufferCache getBufferCache() {
+        return diskInvIndexFactory.getBufferCache();
+    }
+}
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
new file mode 100644
index 0000000..15a1633
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
@@ -0,0 +1,244 @@
+/*
+ * Copyright 2009-2010 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.impls;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+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;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+// TODO: Refactor for better code sharing with other file managers.
+public class LSMInvertedIndexFileManager extends AbstractLSMIndexFileManager implements IInvertedIndexFileNameMapper {
+    private static final String DICT_BTREE_SUFFIX = "b";
+    private static final String INVLISTS_SUFFIX = "i";
+    private static final String DELETED_KEYS_BTREE_SUFFIX = "d";
+
+    // We only need a BTree factory because the inverted indexes consistency is validated against its dictionary BTree.
+    private final BTreeFactory btreeFactory;
+
+    private static FilenameFilter dictBTreeFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".") && name.endsWith(DICT_BTREE_SUFFIX);
+        }
+    };
+
+    private static FilenameFilter deletedKeysBTreeFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".") && name.endsWith(DELETED_KEYS_BTREE_SUFFIX);
+        }
+    };
+
+    public LSMInvertedIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
+            BTreeFactory btreeFactory, int startIODeviceIndex) {
+        super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+        this.btreeFactory = btreeFactory;
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelFlushFileReference() {
+        Date date = new Date();
+        String ts = formatter.format(date);
+        String baseName = baseDir + ts + SPLIT_STRING + ts;
+        // Begin timestamp and end timestamp are identical since it is a flush
+        return new LSMComponentFileReferences(createFlushFile(baseName + SPLIT_STRING + DICT_BTREE_SUFFIX),
+                createFlushFile(baseName + SPLIT_STRING + DELETED_KEYS_BTREE_SUFFIX), createFlushFile(baseName
+                        + SPLIT_STRING + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
+            throws HyracksDataException {
+        String[] firstTimestampRange = firstFileName.split(SPLIT_STRING);
+        String[] lastTimestampRange = lastFileName.split(SPLIT_STRING);
+
+        String baseName = baseDir + firstTimestampRange[0] + SPLIT_STRING + lastTimestampRange[1];
+        // Get the range of timestamps by taking the earliest and the latest timestamps
+        return new LSMComponentFileReferences(createMergeFile(baseName + SPLIT_STRING + DICT_BTREE_SUFFIX),
+                createMergeFile(baseName + SPLIT_STRING + DELETED_KEYS_BTREE_SUFFIX), createMergeFile(baseName
+                        + SPLIT_STRING + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException, IndexException {
+        List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
+        ArrayList<ComparableFileName> allDictBTreeFiles = new ArrayList<ComparableFileName>();
+        ArrayList<ComparableFileName> allDeletedKeysBTreeFiles = new ArrayList<ComparableFileName>();
+        ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
+
+        // Gather files from all IODeviceHandles.
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            cleanupAndGetValidFilesInternal(dev, bloomFilterFilter, null, allBloomFilterFiles);
+            HashSet<String> bloomFilterFilesSet = new HashSet<String>();
+            for (ComparableFileName cmpFileName : allBloomFilterFiles) {
+                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+                bloomFilterFilesSet.add(cmpFileName.fileName.substring(0, index));
+            }
+            // List of valid BTree files that may or may not have a bloom filter buddy. Will check for buddies below.
+            ArrayList<ComparableFileName> tmpAllDeletedBTreeFiles = new ArrayList<ComparableFileName>();
+            cleanupAndGetValidFilesInternal(dev, deletedKeysBTreeFilter, btreeFactory, tmpAllDeletedBTreeFiles);
+
+            // Look for buddy bloom filters for all valid BTrees. 
+            // If no buddy is found, delete the file, otherwise add the BTree to allBTreeFiles. 
+            HashSet<String> deletedKeysBTreeFilesSet = new HashSet<String>();
+            for (ComparableFileName cmpFileName : tmpAllDeletedBTreeFiles) {
+                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+                String file = cmpFileName.fileName.substring(0, index);
+                if (bloomFilterFilesSet.contains(file)) {
+                    allDeletedKeysBTreeFiles.add(cmpFileName);
+                    deletedKeysBTreeFilesSet.add(cmpFileName.fileName.substring(0, index));
+                } else {
+                    // Couldn't find the corresponding BTree file; thus, delete
+                    // the deleted-keys BTree file.
+                    // There is no need to delete the inverted-lists file corresponding to the non-existent
+                    // dictionary BTree, because we flush the dictionary BTree first. So if a dictionary BTree 
+                    // file does not exists, then neither can its inverted-list file.
+                    File invalidDeletedKeysBTreeFile = new File(cmpFileName.fullPath);
+                    invalidDeletedKeysBTreeFile.delete();
+                }
+            }
+
+            // We use the dictionary BTree of the inverted index for validation.
+            // List of valid dictionary BTree files that may or may not have a deleted-keys BTree buddy. Will check for buddies below.
+            ArrayList<ComparableFileName> tmpAllBTreeFiles = new ArrayList<ComparableFileName>();
+            cleanupAndGetValidFilesInternal(dev, dictBTreeFilter, btreeFactory, tmpAllBTreeFiles);
+            // Look for buddy deleted-keys BTrees for all valid dictionary BTrees. 
+            // If no buddy is found, delete the file, otherwise add the dictionary BTree to allBTreeFiles. 
+            for (ComparableFileName cmpFileName : tmpAllBTreeFiles) {
+                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+                String file = cmpFileName.fileName.substring(0, index);
+                if (deletedKeysBTreeFilesSet.contains(file)) {
+                    allDictBTreeFiles.add(cmpFileName);
+                } else {
+                    // Couldn't find the corresponding BTree file; thus, delete
+                    // the deleted-keys BTree file.
+                    // There is no need to delete the inverted-lists file corresponding to the non-existent
+                    // dictionary BTree, because we flush the dictionary BTree first. So if a dictionary BTree 
+                    // file does not exists, then neither can its inverted-list file.
+                    File invalidDeletedKeysBTreeFile = new File(cmpFileName.fullPath);
+                    invalidDeletedKeysBTreeFile.delete();
+                }
+            }
+        }
+        // Sanity check.
+        if (allDictBTreeFiles.size() != allDeletedKeysBTreeFiles.size()
+                || allDictBTreeFiles.size() != allBloomFilterFiles.size()) {
+            throw new HyracksDataException(
+                    "Unequal number of valid Dictionary BTree, Deleted BTree, and Bloom Filter files found. Aborting cleanup.");
+        }
+
+        // Trivial cases.
+        if (allDictBTreeFiles.isEmpty() || allDeletedKeysBTreeFiles.isEmpty() || allBloomFilterFiles.isEmpty()) {
+            return validFiles;
+        }
+
+        if (allDictBTreeFiles.size() == 1 && allDeletedKeysBTreeFiles.size() == 1 && allBloomFilterFiles.size() == 1) {
+            validFiles.add(new LSMComponentFileReferences(allDictBTreeFiles.get(0).fileRef, allDeletedKeysBTreeFiles
+                    .get(0).fileRef, allBloomFilterFiles.get(0).fileRef));
+            return validFiles;
+        }
+
+        // Sorts files names from earliest to latest timestamp.
+        Collections.sort(allDeletedKeysBTreeFiles);
+        Collections.sort(allDictBTreeFiles);
+        Collections.sort(allBloomFilterFiles);
+
+        List<ComparableFileName> validComparableDictBTreeFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastDictBTree = allDictBTreeFiles.get(0);
+        validComparableDictBTreeFiles.add(lastDictBTree);
+
+        List<ComparableFileName> validComparableDeletedKeysBTreeFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastDeletedKeysBTree = allDeletedKeysBTreeFiles.get(0);
+        validComparableDeletedKeysBTreeFiles.add(lastDeletedKeysBTree);
+
+        List<ComparableFileName> validComparableBloomFilterFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastBloomFilter = allBloomFilterFiles.get(0);
+        validComparableBloomFilterFiles.add(lastBloomFilter);
+
+        for (int i = 1; i < allDictBTreeFiles.size(); i++) {
+            ComparableFileName currentRTree = allDictBTreeFiles.get(i);
+            ComparableFileName currentBTree = 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
+                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0) {
+                validComparableDictBTreeFiles.add(currentRTree);
+                validComparableDeletedKeysBTreeFiles.add(currentBTree);
+                validComparableBloomFilterFiles.add(currentBloomFilter);
+                lastDictBTree = currentRTree;
+                lastDeletedKeysBTree = currentBTree;
+                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
+                    && 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 invalidBloomFilterFile = new File(currentBloomFilter.fullPath);
+                invalidBloomFilterFile.delete();
+            } else {
+                // This scenario should not be possible.
+                throw new HyracksDataException("Found LSM files with overlapping but not contained timetamp intervals.");
+            }
+        }
+
+        // Sort valid files in reverse lexicographical order, such that newer
+        // files come first.
+        Collections.sort(validComparableDictBTreeFiles, recencyCmp);
+        Collections.sort(validComparableDeletedKeysBTreeFiles, recencyCmp);
+        Collections.sort(validComparableBloomFilterFiles, recencyCmp);
+
+        Iterator<ComparableFileName> dictBTreeFileIter = validComparableDictBTreeFiles.iterator();
+        Iterator<ComparableFileName> deletedKeysBTreeIter = validComparableDeletedKeysBTreeFiles.iterator();
+        Iterator<ComparableFileName> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
+        while (dictBTreeFileIter.hasNext() && deletedKeysBTreeIter.hasNext()) {
+            ComparableFileName cmpDictBTreeFile = dictBTreeFileIter.next();
+            ComparableFileName cmpDeletedKeysBTreeFile = deletedKeysBTreeIter.next();
+            ComparableFileName cmpBloomFilterFileName = bloomFilterFileIter.next();
+            validFiles.add(new LSMComponentFileReferences(cmpDictBTreeFile.fileRef, cmpDeletedKeysBTreeFile.fileRef,
+                    cmpBloomFilterFileName.fileRef));
+        }
+
+        return validFiles;
+    }
+
+    @Override
+    public String getInvListsFilePath(String dictBTreeFilePath) {
+        int index = dictBTreeFilePath.lastIndexOf(SPLIT_STRING);
+        String file = dictBTreeFilePath.substring(0, index);
+        return file + SPLIT_STRING + INVLISTS_SUFFIX;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java
new file mode 100644
index 0000000..eedf0da
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+
+public class LSMInvertedIndexFlushOperation implements ILSMIOOperation {
+    private final ILSMIndexAccessorInternal accessor;
+    private final LSMInvertedIndexMutableComponent flushingComponent;
+    private final FileReference dictBTreeFlushTarget;
+    private final FileReference deletedKeysBTreeFlushTarget;
+    private final FileReference bloomFilterFlushTarget;
+    private final ILSMIOOperationCallback callback;
+
+    public LSMInvertedIndexFlushOperation(ILSMIndexAccessorInternal accessor,
+            LSMInvertedIndexMutableComponent flushingComponent, FileReference dictBTreeFlushTarget,
+            FileReference deletedKeysBTreeFlushTarget, FileReference bloomFilterFlushTarget,
+            ILSMIOOperationCallback callback) {
+        this.accessor = accessor;
+        this.flushingComponent = flushingComponent;
+        this.dictBTreeFlushTarget = dictBTreeFlushTarget;
+        this.deletedKeysBTreeFlushTarget = deletedKeysBTreeFlushTarget;
+        this.bloomFilterFlushTarget = bloomFilterFlushTarget;
+        this.callback = callback;
+    }
+
+    @Override
+    public Set<IODeviceHandle> getReadDevices() {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public Set<IODeviceHandle> getWriteDevices() {
+        Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+        devs.add(dictBTreeFlushTarget.getDeviceHandle());
+        devs.add(deletedKeysBTreeFlushTarget.getDeviceHandle());
+        devs.add(bloomFilterFlushTarget.getDeviceHandle());
+        return devs;
+
+    }
+
+    @Override
+    public void perform() throws HyracksDataException, IndexException {
+        accessor.flush(this);
+    }
+
+    @Override
+    public ILSMIOOperationCallback getCallback() {
+        return callback;
+    }
+
+    public FileReference getDictBTreeFlushTarget() {
+        return dictBTreeFlushTarget;
+    }
+
+    public FileReference getDeletedKeysBTreeFlushTarget() {
+        return deletedKeysBTreeFlushTarget;
+    }
+
+    public FileReference getBloomFilterFlushTarget() {
+        return bloomFilterFlushTarget;
+    }
+
+    public LSMInvertedIndexMutableComponent getFlushingComponent() {
+        return flushingComponent;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexImmutableComponent.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexImmutableComponent.java
new file mode 100644
index 0000000..4c9b5e8
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexImmutableComponent.java
@@ -0,0 +1,42 @@
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractImmutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+
+public class LSMInvertedIndexImmutableComponent extends AbstractImmutableLSMComponent {
+
+    private final IInvertedIndex invIndex;
+    private final BTree deletedKeysBTree;
+    private final BloomFilter bloomFilter;
+
+    public LSMInvertedIndexImmutableComponent(IInvertedIndex invIndex, BTree deletedKeysBTree, BloomFilter bloomFilter) {
+        this.invIndex = invIndex;
+        this.deletedKeysBTree = deletedKeysBTree;
+        this.bloomFilter = bloomFilter;
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        invIndex.deactivate();
+        invIndex.destroy();
+        deletedKeysBTree.deactivate();
+        deletedKeysBTree.destroy();
+        bloomFilter.deactivate();
+        bloomFilter.destroy();
+    }
+
+    public IInvertedIndex getInvIndex() {
+        return invIndex;
+    }
+
+    public BTree getDeletedKeysBTree() {
+        return deletedKeysBTree;
+    }
+
+    public BloomFilter getBloomFilter() {
+        return bloomFilter;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java
new file mode 100644
index 0000000..dea628c
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
+
+public class LSMInvertedIndexMergeOperation implements ILSMIOOperation {
+    private final ILSMIndexAccessorInternal accessor;
+    private final List<ILSMComponent> mergingComponents;
+    private final IIndexCursor cursor;
+    private final FileReference dictBTreeMergeTarget;
+    private final FileReference deletedKeysBTreeMergeTarget;
+    private final FileReference bloomFilterMergeTarget;
+    private final ILSMIOOperationCallback callback;
+
+    public LSMInvertedIndexMergeOperation(ILSMIndexAccessorInternal accessor, List<ILSMComponent> mergingComponents,
+            IIndexCursor cursor, FileReference dictBTreeMergeTarget, FileReference deletedKeysBTreeMergeTarget,
+            FileReference bloomFilterMergeTarget, ILSMIOOperationCallback callback) {
+        this.accessor = accessor;
+        this.mergingComponents = mergingComponents;
+        this.cursor = cursor;
+        this.dictBTreeMergeTarget = dictBTreeMergeTarget;
+        this.deletedKeysBTreeMergeTarget = deletedKeysBTreeMergeTarget;
+        this.bloomFilterMergeTarget = bloomFilterMergeTarget;
+        this.callback = callback;
+    }
+
+    @Override
+    public Set<IODeviceHandle> getReadDevices() {
+        Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+        for (Object o : mergingComponents) {
+            LSMInvertedIndexImmutableComponent component = (LSMInvertedIndexImmutableComponent) o;
+            OnDiskInvertedIndex invIndex = (OnDiskInvertedIndex) component.getInvIndex();
+            devs.add(invIndex.getBTree().getFileReference().getDeviceHandle());
+            devs.add(component.getDeletedKeysBTree().getFileReference().getDeviceHandle());
+            devs.add(component.getBloomFilter().getFileReference().getDeviceHandle());
+        }
+        return devs;
+    }
+
+    @Override
+    public Set<IODeviceHandle> getWriteDevices() {
+        Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+        devs.add(dictBTreeMergeTarget.getDeviceHandle());
+        devs.add(deletedKeysBTreeMergeTarget.getDeviceHandle());
+        devs.add(bloomFilterMergeTarget.getDeviceHandle());
+        return devs;
+    }
+
+    @Override
+    public void perform() throws HyracksDataException, IndexException {
+        accessor.merge(this);
+    }
+
+    @Override
+    public ILSMIOOperationCallback getCallback() {
+        return callback;
+    }
+
+    public FileReference getDictBTreeMergeTarget() {
+        return dictBTreeMergeTarget;
+    }
+
+    public FileReference getDeletedKeysBTreeMergeTarget() {
+        return deletedKeysBTreeMergeTarget;
+    }
+
+    public FileReference getBloomFilterMergeTarget() {
+        return bloomFilterMergeTarget;
+    }
+
+    public IIndexCursor getCursor() {
+        return cursor;
+    }
+
+    public List<ILSMComponent> getMergingComponents() {
+        return mergingComponents;
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMutableComponent.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMutableComponent.java
new file mode 100644
index 0000000..c36319d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMutableComponent.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+
+public class LSMInvertedIndexMutableComponent extends AbstractMutableLSMComponent {
+
+    private final IInvertedIndex invIndex;
+    private final BTree deletedKeysBTree;
+    private final IInMemoryFreePageManager mfpm;
+
+    public LSMInvertedIndexMutableComponent(IInvertedIndex invIndex, BTree deletedKeysBTree,
+            IInMemoryFreePageManager mfpm) {
+        this.invIndex = invIndex;
+        this.deletedKeysBTree = deletedKeysBTree;
+        this.mfpm = mfpm;
+    }
+
+    public IInvertedIndex getInvIndex() {
+        return invIndex;
+    }
+
+    public BTree getDeletedKeysBTree() {
+        return deletedKeysBTree;
+    }
+
+    @Override
+    protected boolean isFull() {
+        return mfpm.isFull();
+    }
+
+    @Override
+    protected void reset() throws HyracksDataException {
+        invIndex.clear();
+        deletedKeysBTree.clear();
+    }
+}
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
new file mode 100644
index 0000000..b961b7a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import java.util.LinkedList;
+import java.util.List;
+
+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;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+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.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
+
+public class LSMInvertedIndexOpContext implements ILSMIndexOperationContext {
+
+    private static final int NUM_DOCUMENT_FIELDS = 1;
+
+    private IndexOperation op;
+    private final IInvertedIndex memInvIndex;
+    private final IIndex memDeletedKeysBTree;
+    private final List<ILSMComponent> componentHolder;
+
+    public final IModificationOperationCallback modificationCallback;
+    public final ISearchOperationCallback searchCallback;
+
+    // Tuple that only has the inverted-index elements (aka keys), projecting away the document fields.
+    public PermutingTupleReference keysOnlyTuple;
+
+    // Accessor to the in-memory inverted index.
+    public IInvertedIndexAccessor memInvIndexAccessor;
+    // Accessor to the deleted-keys BTree.
+    public IIndexAccessor deletedKeysBTreeAccessor;
+
+    public LSMInvertedIndexOpContext(IInvertedIndex memInvIndex, IIndex memDeletedKeysBTree,
+            IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback) {
+        this.memInvIndex = memInvIndex;
+        this.memDeletedKeysBTree = memDeletedKeysBTree;
+        this.componentHolder = new LinkedList<ILSMComponent>();
+        this.modificationCallback = modificationCallback;
+        this.searchCallback = searchCallback;
+    }
+
+    @Override
+    public void reset() {
+        componentHolder.clear();
+    }
+
+    @Override
+    // TODO: Ignore opcallback for now.
+    public void setOperation(IndexOperation newOp) {
+        reset();
+        switch (newOp) {
+            case INSERT:
+            case DELETE:
+            case PHYSICALDELETE: {
+                if (deletedKeysBTreeAccessor == null) {
+                    memInvIndexAccessor = (IInvertedIndexAccessor) memInvIndex.createAccessor(
+                            NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+                    deletedKeysBTreeAccessor = memDeletedKeysBTree.createAccessor(NoOpOperationCallback.INSTANCE,
+                            NoOpOperationCallback.INSTANCE);
+                    // Project away the document fields, leaving only the key fields.
+                    int numKeyFields = memInvIndex.getInvListTypeTraits().length;
+                    int[] keyFieldPermutation = new int[numKeyFields];
+                    for (int i = 0; i < numKeyFields; i++) {
+                        keyFieldPermutation[i] = NUM_DOCUMENT_FIELDS + i;
+                    }
+                    keysOnlyTuple = new PermutingTupleReference(keyFieldPermutation);
+                }
+                break;
+            }
+        }
+        op = newOp;
+    }
+
+    @Override
+    public IndexOperation getOperation() {
+        return op;
+    }
+
+    @Override
+    public List<ILSMComponent> getComponentHolder() {
+        return componentHolder;
+    }
+
+    @Override
+    public ISearchOperationCallback getSearchOperationCallback() {
+        return searchCallback;
+    }
+
+    @Override
+    public IModificationOperationCallback getModificationCallback() {
+        return modificationCallback;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
new file mode 100644
index 0000000..1b5949a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
@@ -0,0 +1,162 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
+
+public class LSMInvertedIndexRangeSearchCursor extends LSMIndexSearchCursor {
+
+    // Assuming the cursor for all deleted-keys indexes are of the same type.
+    private IIndexCursor[] deletedKeysBTreeCursors;
+    protected ArrayList<IIndexAccessor> deletedKeysBTreeAccessors;
+    protected PermutingTupleReference keysOnlyTuple;
+    protected RangePredicate keySearchPred;
+
+    public LSMInvertedIndexRangeSearchCursor(ILSMIndexOperationContext opCtx) {
+        super(opCtx);
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        super.next();
+    }
+
+    @Override
+    public void open(ICursorInitialState initState, ISearchPredicate searchPred) throws IndexException,
+            HyracksDataException {
+        LSMInvertedIndexRangeSearchCursorInitialState lsmInitState = (LSMInvertedIndexRangeSearchCursorInitialState) initState;
+        cmp = lsmInitState.getOriginalKeyComparator();
+        int numComponents = lsmInitState.getNumComponents();
+        rangeCursors = new IIndexCursor[numComponents];
+        for (int i = 0; i < numComponents; i++) {
+            IInvertedIndexAccessor invIndexAccessor = (IInvertedIndexAccessor) lsmInitState.getIndexAccessors().get(i);
+            rangeCursors[i] = invIndexAccessor.createRangeSearchCursor();
+            invIndexAccessor.rangeSearch(rangeCursors[i], lsmInitState.getSearchPredicate());
+        }
+        lsmHarness = lsmInitState.getLSMHarness();
+        operationalComponents = lsmInitState.getOperationalComponents();
+        includeMemComponent = lsmInitState.getIncludeMemComponent();
+
+        // For searching the deleted-keys BTrees.
+        this.keysOnlyTuple = lsmInitState.getKeysOnlyTuple();
+        deletedKeysBTreeAccessors = lsmInitState.getDeletedKeysBTreeAccessors();
+
+        if (!deletedKeysBTreeAccessors.isEmpty()) {
+            deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
+            int i = 0;
+            if (includeMemComponent) {
+                // No need for a bloom filter for the in-memory BTree.
+                deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor();
+                ++i;
+            }
+            for (; i < deletedKeysBTreeCursors.length; i++) {
+                deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitState
+                        .getgetDeletedKeysBTreeLeafFrameFactory().createFrame(), false,
+                        ((LSMInvertedIndexImmutableComponent) operationalComponents.get(i)).getBloomFilter());
+            }
+
+        }
+        MultiComparator keyCmp = lsmInitState.getKeyComparator();
+        keySearchPred = new RangePredicate(keysOnlyTuple, keysOnlyTuple, true, true, keyCmp, keyCmp);
+
+        setPriorityQueueComparator();
+        initPriorityQueue();
+    }
+
+    /**
+     * Check deleted-keys BTrees whether they contain the key in the checkElement's tuple.
+     */
+    @Override
+    protected boolean isDeleted(PriorityQueueElement checkElement) throws HyracksDataException, IndexException {
+        keysOnlyTuple.reset(checkElement.getTuple());
+        int end = checkElement.getCursorIndex();
+        for (int i = 0; i < end; i++) {
+            deletedKeysBTreeCursors[i].reset();
+            try {
+                deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], keySearchPred);
+                if (deletedKeysBTreeCursors[i].hasNext()) {
+                    return true;
+                }
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            } finally {
+                deletedKeysBTreeCursors[i].close();
+            }
+        }
+        return false;
+    }
+    
+    @Override
+    protected void checkPriorityQueue() throws HyracksDataException, IndexException {
+        while (!outputPriorityQueue.isEmpty() || needPush == true) {
+            if (!outputPriorityQueue.isEmpty()) {
+                PriorityQueueElement checkElement = outputPriorityQueue.peek();
+                // If there is no previous tuple or the previous tuple can be ignored
+                if (outputElement == null) {
+                    if (isDeleted(checkElement)) {
+                        // If the key has been deleted then pop it and set needPush to true.
+                        // We cannot push immediately because the tuple may be
+                        // modified if hasNext() is called
+                        outputElement = outputPriorityQueue.poll();
+                        needPush = true;
+                    } else {
+                        break;
+                    }
+                } else {
+                    // Compare the previous tuple and the head tuple in the PQ
+                    if (compare(cmp, outputElement.getTuple(), checkElement.getTuple()) == 0) {
+                        // If the previous tuple and the head tuple are
+                        // identical
+                        // then pop the head tuple and push the next tuple from
+                        // the tree of head tuple
+
+                        // the head element of PQ is useless now
+                        PriorityQueueElement e = outputPriorityQueue.poll();
+                        pushIntoPriorityQueue(e);
+                    } else {
+                        // If the previous tuple and the head tuple are different
+                        // the info of previous tuple is useless
+                        if (needPush == true) {
+                            pushIntoPriorityQueue(outputElement);
+                            needPush = false;
+                        }
+                        outputElement = null;
+                    }
+                }
+            } else {
+                // the priority queue is empty and needPush
+                pushIntoPriorityQueue(outputElement);
+                needPush = false;
+                outputElement = null;
+            }
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursorInitialState.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursorInitialState.java
new file mode 100644
index 0000000..0cec92e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursorInitialState.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public class LSMInvertedIndexRangeSearchCursorInitialState implements ICursorInitialState {
+
+    private final MultiComparator tokensAndKeysCmp;
+    private final MultiComparator keyCmp;
+    private final ILSMHarness lsmHarness;
+
+    private final ArrayList<IIndexAccessor> indexAccessors;
+    private final ArrayList<IIndexAccessor> deletedKeysBTreeAccessors;
+    private final ISearchPredicate predicate;
+    private final PermutingTupleReference keysOnlyTuple;
+    private final ITreeIndexFrameFactory deletedKeysBtreeLeafFrameFactory;
+
+    private final boolean includeMemComponent;
+    private final List<ILSMComponent> operationalComponents;
+
+    public LSMInvertedIndexRangeSearchCursorInitialState(MultiComparator tokensAndKeysCmp, MultiComparator keyCmp,
+            PermutingTupleReference keysOnlyTuple, ITreeIndexFrameFactory deletedKeysBtreeLeafFrameFactory,
+            boolean includeMemComponent, ILSMHarness lsmHarness, ArrayList<IIndexAccessor> indexAccessors,
+            ArrayList<IIndexAccessor> deletedKeysBTreeAccessors, ISearchPredicate predicate,
+            List<ILSMComponent> operationalComponents) {
+        this.tokensAndKeysCmp = tokensAndKeysCmp;
+        this.keyCmp = keyCmp;
+        this.keysOnlyTuple = keysOnlyTuple;
+        this.deletedKeysBtreeLeafFrameFactory = deletedKeysBtreeLeafFrameFactory;
+        this.lsmHarness = lsmHarness;
+        this.indexAccessors = indexAccessors;
+        this.deletedKeysBTreeAccessors = deletedKeysBTreeAccessors;
+        this.predicate = predicate;
+        this.includeMemComponent = includeMemComponent;
+        this.operationalComponents = operationalComponents;
+    }
+
+    public int getNumComponents() {
+        return indexAccessors.size();
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        return null;
+    }
+
+    @Override
+    public void setPage(ICachedPage page) {
+    }
+
+    public List<ILSMComponent> getOperationalComponents() {
+        return operationalComponents;
+    }
+
+    public ILSMHarness getLSMHarness() {
+        return lsmHarness;
+    }
+
+    @Override
+    public ISearchOperationCallback getSearchOperationCallback() {
+        return null;
+    }
+
+    @Override
+    public void setSearchOperationCallback(ISearchOperationCallback searchCallback) {
+        // Do nothing.
+    }
+
+    public ArrayList<IIndexAccessor> getIndexAccessors() {
+        return indexAccessors;
+    }
+
+    public ArrayList<IIndexAccessor> getDeletedKeysBTreeAccessors() {
+        return deletedKeysBTreeAccessors;
+    }
+
+    public ISearchPredicate getSearchPredicate() {
+        return predicate;
+    }
+
+    @Override
+    public void setOriginialKeyComparator(MultiComparator originalCmp) {
+        // Do nothing.
+    }
+
+    @Override
+    public MultiComparator getOriginalKeyComparator() {
+        return tokensAndKeysCmp;
+    }
+
+    public MultiComparator getKeyComparator() {
+        return keyCmp;
+    }
+
+    public ITreeIndexFrameFactory getgetDeletedKeysBTreeLeafFrameFactory() {
+        return deletedKeysBtreeLeafFrameFactory;
+    }
+
+    public boolean getIncludeMemComponent() {
+        return includeMemComponent;
+    }
+
+    public PermutingTupleReference getKeysOnlyTuple() {
+        return keysOnlyTuple;
+    }
+}
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
new file mode 100644
index 0000000..36ad51b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -0,0 +1,186 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException;
+
+/**
+ * Searches the components one-by-one, completely consuming a cursor before moving on to the next one.
+ * Therefore, the are no guarantees about sort order of the results.
+ */
+public class LSMInvertedIndexSearchCursor implements IIndexCursor {
+
+    private IIndexAccessor currentAccessor;
+    private IIndexCursor currentCursor;
+    private int accessorIndex = -1;
+    private boolean tupleConsumed = true;
+    private ILSMHarness harness;
+    private boolean includeMemComponent;
+    private List<IIndexAccessor> indexAccessors;
+    private ISearchPredicate searchPred;
+    private ISearchOperationCallback searchCallback;
+
+    // Assuming the cursor for all deleted-keys indexes are of the same type.
+    private IIndexCursor[] deletedKeysBTreeCursors;
+    private List<IIndexAccessor> deletedKeysBTreeAccessors;
+    private RangePredicate keySearchPred;
+    private ILSMIndexOperationContext opCtx;
+
+    private List<ILSMComponent> operationalComponents;
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        LSMInvertedIndexSearchCursorInitialState lsmInitState = (LSMInvertedIndexSearchCursorInitialState) initialState;
+        harness = lsmInitState.getLSMHarness();
+        includeMemComponent = lsmInitState.getIncludeMemComponent();
+        operationalComponents = lsmInitState.getOperationalComponents();
+        indexAccessors = lsmInitState.getIndexAccessors();
+        opCtx = lsmInitState.getOpContext();
+        accessorIndex = 0;
+        this.searchPred = searchPred;
+        this.searchCallback = lsmInitState.getSearchOperationCallback();
+
+        // For searching the deleted-keys BTrees.
+        deletedKeysBTreeAccessors = lsmInitState.getDeletedKeysBTreeAccessors();
+        deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
+        int i = 0;
+        if (includeMemComponent) {
+            // No need for a bloom filter for the in-memory BTree.
+            deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor();
+            ++i;
+        }
+        for (; i < deletedKeysBTreeCursors.length; i++) {
+            deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitState
+                    .getgetDeletedKeysBTreeLeafFrameFactory().createFrame(), false,
+                    ((LSMInvertedIndexImmutableComponent) operationalComponents.get(i)).getBloomFilter());
+        }
+
+        MultiComparator keyCmp = lsmInitState.getKeyComparator();
+        keySearchPred = new RangePredicate(null, null, true, true, keyCmp, keyCmp);
+    }
+
+    protected boolean isDeleted(ITupleReference key) throws HyracksDataException, IndexException {
+        keySearchPred.setLowKey(key, true);
+        keySearchPred.setHighKey(key, true);
+        for (int i = 0; i < accessorIndex; i++) {
+            deletedKeysBTreeCursors[i].reset();
+            try {
+                deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], keySearchPred);
+                if (deletedKeysBTreeCursors[i].hasNext()) {
+                    return true;
+                }
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            } finally {
+                deletedKeysBTreeCursors[i].close();
+            }
+        }
+        return false;
+    }
+
+    // Move to the next tuple that has not been deleted.
+    private boolean nextValidTuple() throws HyracksDataException, IndexException {
+        while (currentCursor.hasNext()) {
+            currentCursor.next();
+            if (!isDeleted(currentCursor.getTuple())) {
+                tupleConsumed = false;
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        if (!tupleConsumed) {
+            return true;
+        }
+        if (currentCursor != null) {
+            if (nextValidTuple()) {
+                return true;
+            }
+            currentCursor.close();
+            accessorIndex++;
+        }
+        while (accessorIndex < indexAccessors.size()) {
+            // Current cursor has been exhausted, switch to next accessor/cursor.
+            currentAccessor = indexAccessors.get(accessorIndex);
+            currentCursor = currentAccessor.createSearchCursor();
+            try {
+                currentAccessor.search(currentCursor, searchPred);
+            } catch (OccurrenceThresholdPanicException e) {
+                throw e;
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+            if (nextValidTuple()) {
+                return true;
+            }
+            // Close as we go to release resources.
+            currentCursor.close();
+            accessorIndex++;
+        }
+        return false;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        // Mark the tuple as consumed, so hasNext() can move on.
+        tupleConsumed = true;
+        // We assume that the underlying cursors materialize their results such that
+        // there is no need to reposition the result cursor after reconciliation.
+        if (!searchCallback.proceed(currentCursor.getTuple())) {
+            searchCallback.reconcile(currentCursor.getTuple());
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        reset();
+        accessorIndex = -1;
+        harness.endSearch(opCtx);
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        if (currentCursor != null) {
+            currentCursor.close();
+            currentCursor = null;
+        }
+        accessorIndex = 0;
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return currentCursor.getTuple();
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursorInitialState.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursorInitialState.java
new file mode 100644
index 0000000..eb6f338
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursorInitialState.java
@@ -0,0 +1,128 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public class LSMInvertedIndexSearchCursorInitialState implements ICursorInitialState {
+
+    private final boolean includeMemComponent;
+    private final ILSMHarness lsmHarness;
+    private final List<IIndexAccessor> indexAccessors;
+    private final List<IIndexAccessor> deletedKeysBTreeAccessors;
+    private final LSMInvertedIndexOpContext ctx;
+    private ISearchOperationCallback searchCallback;
+    private MultiComparator originalCmp;
+    private final MultiComparator keyCmp;
+    private final PermutingTupleReference keysOnlyTuple;
+    private final ITreeIndexFrameFactory deletedKeysBtreeLeafFrameFactory;
+
+    private final List<ILSMComponent> operationalComponents;
+
+    public LSMInvertedIndexSearchCursorInitialState(final MultiComparator keyCmp,
+            PermutingTupleReference keysOnlyTuple, List<IIndexAccessor> indexAccessors,
+            List<IIndexAccessor> deletedKeysBTreeAccessors, ITreeIndexFrameFactory deletedKeysBtreeLeafFrameFactory,
+            IIndexOperationContext ctx, boolean includeMemComponent, ILSMHarness lsmHarness,
+            List<ILSMComponent> operationalComponents) {
+        this.keyCmp = keyCmp;
+        this.keysOnlyTuple = keysOnlyTuple;
+        this.indexAccessors = indexAccessors;
+        this.deletedKeysBTreeAccessors = deletedKeysBTreeAccessors;
+        this.deletedKeysBtreeLeafFrameFactory = deletedKeysBtreeLeafFrameFactory;
+        this.includeMemComponent = includeMemComponent;
+        this.operationalComponents = operationalComponents;
+        this.lsmHarness = lsmHarness;
+        this.ctx = (LSMInvertedIndexOpContext) ctx;
+        this.searchCallback = this.ctx.searchCallback;
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        return null;
+    }
+
+    @Override
+    public void setPage(ICachedPage page) {
+    }
+
+    public List<ILSMComponent> getOperationalComponents() {
+        return operationalComponents;
+    }
+
+    public List<IIndexAccessor> getIndexAccessors() {
+        return indexAccessors;
+    }
+
+    public boolean getIncludeMemComponent() {
+        return includeMemComponent;
+    }
+
+    public ILSMHarness getLSMHarness() {
+        return lsmHarness;
+    }
+
+    public ILSMIndexOperationContext getOpContext() {
+        return ctx;
+    }
+
+    @Override
+    public ISearchOperationCallback getSearchOperationCallback() {
+        return searchCallback;
+    }
+
+    @Override
+    public void setSearchOperationCallback(ISearchOperationCallback searchCallback) {
+        this.searchCallback = searchCallback;
+    }
+
+    @Override
+    public MultiComparator getOriginalKeyComparator() {
+        return originalCmp;
+    }
+
+    @Override
+    public void setOriginialKeyComparator(MultiComparator originalCmp) {
+        this.originalCmp = originalCmp;
+    }
+
+    public MultiComparator getKeyComparator() {
+        return keyCmp;
+    }
+
+    public List<IIndexAccessor> getDeletedKeysBTreeAccessors() {
+        return deletedKeysBTreeAccessors;
+    }
+    
+    public ITreeIndexFrameFactory getgetDeletedKeysBTreeLeafFrameFactory() {
+        return deletedKeysBtreeLeafFrameFactory;
+    }
+
+    public PermutingTupleReference getKeysOnlyTuple() {
+        return keysOnlyTuple;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
new file mode 100644
index 0000000..1b293eb
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2012 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.impls;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
+
+    public PartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, OnDiskInvertedIndexFactory diskInvIndexFactory,
+            BTreeFactory deletedKeysBTreeFactory, BloomFilterFactory bloomFilterFactory,
+            ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+            throws IndexException {
+        super(memBufferCache, memFreePageManager, diskInvIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory,
+                fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                tokenCmpFactories, tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+    }
+
+    protected InMemoryInvertedIndex createInMemoryInvertedIndex(IInMemoryBufferCache memBufferCache)
+            throws IndexException {
+        return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(memBufferCache, memFreePageManager,
+                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
+    }
+
+}
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
new file mode 100644
index 0000000..d5a074e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+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.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeNonExistentKeyException;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public class InMemoryInvertedIndex implements IInvertedIndex {
+
+    protected final BTree btree;
+    protected final FileReference memBTreeFile = new FileReference(new File("memBTree"));
+    protected final ITypeTraits[] tokenTypeTraits;
+    protected final IBinaryComparatorFactory[] tokenCmpFactories;
+    protected final ITypeTraits[] invListTypeTraits;
+    protected final IBinaryComparatorFactory[] invListCmpFactories;
+    protected final IBinaryTokenizerFactory tokenizerFactory;
+
+    protected final ITypeTraits[] btreeTypeTraits;
+    protected final IBinaryComparatorFactory[] btreeCmpFactories;
+
+    public InMemoryInvertedIndex(IBufferCache memBufferCache, IFreePageManager memFreePageManager,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory) throws BTreeException {
+        this.tokenTypeTraits = tokenTypeTraits;
+        this.tokenCmpFactories = tokenCmpFactories;
+        this.invListTypeTraits = invListTypeTraits;
+        this.invListCmpFactories = invListCmpFactories;
+        this.tokenizerFactory = tokenizerFactory;
+        // BTree tuples: <tokens, inverted-list elements>.
+        int numBTreeFields = tokenTypeTraits.length + invListTypeTraits.length;
+        btreeTypeTraits = new ITypeTraits[numBTreeFields];
+        btreeCmpFactories = new IBinaryComparatorFactory[numBTreeFields];
+        for (int i = 0; i < tokenTypeTraits.length; i++) {
+            btreeTypeTraits[i] = tokenTypeTraits[i];
+            btreeCmpFactories[i] = tokenCmpFactories[i];
+        }
+        for (int i = 0; i < invListTypeTraits.length; i++) {
+            btreeTypeTraits[tokenTypeTraits.length + i] = invListTypeTraits[i];
+            btreeCmpFactories[tokenTypeTraits.length + i] = invListCmpFactories[i];
+        }
+        this.btree = BTreeUtils.createBTree(memBufferCache, memFreePageManager,
+                ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), btreeTypeTraits, btreeCmpFactories,
+                BTreeLeafFrameType.REGULAR_NSM, memBTreeFile);
+    }
+
+    @Override
+    public void create() throws HyracksDataException {
+        btree.create();
+    }
+
+    @Override
+    public void activate() throws HyracksDataException {
+        btree.activate();
+    }
+
+    @Override
+    public void clear() throws HyracksDataException {
+        btree.clear();
+    }
+
+    @Override
+    public void deactivate() throws HyracksDataException {
+        btree.deactivate();
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        btree.destroy();
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        btree.validate();
+    }
+
+    public void insert(ITupleReference tuple, BTreeAccessor btreeAccessor, IIndexOperationContext ictx)
+            throws HyracksDataException, IndexException {
+        InMemoryInvertedIndexOpContext ctx = (InMemoryInvertedIndexOpContext) ictx;
+        ctx.tupleIter.reset(tuple);
+        while (ctx.tupleIter.hasNext()) {
+            ctx.tupleIter.next();
+            ITupleReference insertTuple = ctx.tupleIter.getTuple();
+            try {
+                btreeAccessor.insert(insertTuple);
+            } catch (BTreeDuplicateKeyException e) {
+                // This exception may be caused by duplicate tokens in the same insert "document".
+                // We ignore such duplicate tokens in all inverted-index implementations, hence
+                // we can safely ignore this exception.
+            }
+        }
+    }
+
+    public void delete(ITupleReference tuple, BTreeAccessor btreeAccessor, IIndexOperationContext ictx)
+            throws HyracksDataException, IndexException {
+        InMemoryInvertedIndexOpContext ctx = (InMemoryInvertedIndexOpContext) ictx;
+        ctx.tupleIter.reset(tuple);
+        while (ctx.tupleIter.hasNext()) {
+            ctx.tupleIter.next();
+            ITupleReference deleteTuple = ctx.tupleIter.getTuple();
+            try {
+                btreeAccessor.delete(deleteTuple);
+            } catch (BTreeNonExistentKeyException e) {
+                // Ignore this exception, since a document may have duplicate tokens.
+            }
+        }
+    }
+
+    @Override
+    public long getMemoryAllocationSize() {
+        InMemoryBufferCache memBufferCache = (InMemoryBufferCache) btree.getBufferCache();
+        return memBufferCache.getNumPages() * memBufferCache.getPageSize();
+    }
+
+    @Override
+    public IInvertedListCursor createInvertedListCursor() {
+        return new InMemoryInvertedListCursor(invListTypeTraits.length, tokenTypeTraits.length);
+    }
+
+    @Override
+    public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey,
+            IIndexOperationContext ictx) throws HyracksDataException, IndexException {
+        InMemoryInvertedIndexOpContext ctx = (InMemoryInvertedIndexOpContext) ictx;
+        ctx.setOperation(IndexOperation.SEARCH);
+        InMemoryInvertedListCursor inMemListCursor = (InMemoryInvertedListCursor) listCursor;
+        inMemListCursor.prepare(ctx.btreeAccessor, ctx.btreePred, ctx.tokenFieldsCmp, ctx.btreeCmp);
+        inMemListCursor.reset(searchKey);
+    }
+
+    @Override
+    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new InMemoryInvertedIndexAccessor(this, new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories,
+                tokenizerFactory));
+    }
+
+    @Override
+    public IBufferCache getBufferCache() {
+        return btree.getBufferCache();
+    }
+
+    public BTree getBTree() {
+        return btree;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getInvListCmpFactories() {
+        return invListCmpFactories;
+    }
+
+    @Override
+    public ITypeTraits[] getInvListTypeTraits() {
+        return invListTypeTraits;
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+            throws IndexException {
+        throw new UnsupportedOperationException("Bulk load not supported by in-memory inverted index.");
+    }
+
+    @Override
+    public ITypeTraits[] getTokenTypeTraits() {
+        return tokenTypeTraits;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getTokenCmpFactories() {
+        return tokenCmpFactories;
+    }
+}
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
new file mode 100644
index 0000000..a62aaf1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
@@ -0,0 +1,116 @@
+/*
+ * Copyright 2009-2010 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.inmemory;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex.DefaultHyracksCommonContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexSearchCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.TOccurrenceSearcher;
+
+public class InMemoryInvertedIndexAccessor implements IInvertedIndexAccessor {
+    // TODO: This ctx needs to go away.
+    protected final IHyracksCommonContext hyracksCtx = new DefaultHyracksCommonContext();
+    protected final IInvertedIndexSearcher searcher;
+    protected IIndexOperationContext opCtx;
+    protected InMemoryInvertedIndex index;
+    protected BTreeAccessor btreeAccessor;
+
+    public InMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx) {
+        this.opCtx = opCtx;
+        this.index = index;
+        this.searcher = createSearcher();
+        this.btreeAccessor = (BTreeAccessor) index.getBTree().createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+    }
+
+    @Override
+    public void insert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        opCtx.setOperation(IndexOperation.INSERT);
+        index.insert(tuple, btreeAccessor, opCtx);
+    }
+
+    @Override
+    public void delete(ITupleReference tuple) throws HyracksDataException, IndexException {
+        opCtx.setOperation(IndexOperation.DELETE);
+        index.delete(tuple, btreeAccessor, opCtx);
+    }
+
+    @Override
+    public IIndexCursor createSearchCursor() {
+        return new OnDiskInvertedIndexSearchCursor(searcher, index.getInvListTypeTraits().length);
+    }
+
+    @Override
+    public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException, IndexException {
+        searcher.search((OnDiskInvertedIndexSearchCursor) cursor, (InvertedIndexSearchPredicate) searchPred, opCtx);
+    }
+
+    @Override
+    public IInvertedListCursor createInvertedListCursor() {
+        return index.createInvertedListCursor();
+    }
+
+    @Override
+    public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey)
+            throws HyracksDataException, IndexException {
+        index.openInvertedListCursor(listCursor, searchKey, opCtx);
+    }
+
+    @Override
+    public IIndexCursor createRangeSearchCursor() {
+        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) index.getBTree().getLeafFrameFactory().createFrame();
+        return new BTreeRangeSearchCursor(leafFrame, false);
+    }
+
+    @Override
+    public void rangeSearch(IIndexCursor cursor, ISearchPredicate searchPred) throws IndexException,
+            HyracksDataException {
+        btreeAccessor.search(cursor, searchPred);
+    }
+
+    public BTreeAccessor getBTreeAccessor() {
+        return btreeAccessor;
+    }
+
+    @Override
+    public void update(ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Update not supported by in-memory inverted index.");
+    }
+
+    @Override
+    public void upsert(ITupleReference tuple) throws HyracksDataException, IndexException {
+        throw new UnsupportedOperationException("Upsert not supported by in-memory inverted index.");
+    }
+
+    protected IInvertedIndexSearcher createSearcher() {
+        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/InMemoryInvertedIndexOpContext.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
new file mode 100644
index 0000000..9c8865e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexTokenizingTupleIterator;
+
+public class InMemoryInvertedIndexOpContext implements IIndexOperationContext {
+    public IndexOperation op;
+    public final BTree btree;
+
+    // Needed for search operations,    
+    public RangePredicate btreePred;
+    public BTreeAccessor btreeAccessor;
+    public MultiComparator btreeCmp;
+    public IBinaryComparatorFactory[] tokenCmpFactories;
+    public MultiComparator tokenFieldsCmp;
+
+    // To generate in-memory BTree tuples for insertions.
+    protected final IBinaryTokenizerFactory tokenizerFactory;
+    public InvertedIndexTokenizingTupleIterator tupleIter;
+
+    public InMemoryInvertedIndexOpContext(BTree btree, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory) {
+        this.btree = btree;
+        this.tokenCmpFactories = tokenCmpFactories;
+        this.tokenizerFactory = tokenizerFactory;
+    }
+
+    @Override
+    public void setOperation(IndexOperation newOp) {
+        switch (newOp) {
+            case INSERT:
+            case DELETE: {
+                if (tupleIter == null) {
+                    setTokenizingTupleIterator();
+                }
+                break;
+            }
+            case SEARCH: {
+                if (btreePred == null) {
+                    btreePred = new RangePredicate(null, null, true, true, null, null);
+                    btreeAccessor = (BTreeAccessor) btree.createAccessor(NoOpOperationCallback.INSTANCE,
+                            NoOpOperationCallback.INSTANCE);
+                    btreeCmp = MultiComparator.createIgnoreFieldLength(btree.getComparatorFactories());
+                    tokenFieldsCmp = MultiComparator.createIgnoreFieldLength(tokenCmpFactories);
+                }
+                break;
+            }
+            default: {
+                throw new UnsupportedOperationException("Unsupported operation " + newOp);
+            }
+        }
+        op = newOp;
+    }
+
+    @Override
+    public void reset() {
+        op = null;
+    }
+
+    @Override
+    public IndexOperation getOperation() {
+        return op;
+    }
+
+    protected void setTokenizingTupleIterator() {
+        IBinaryTokenizer tokenizer = tokenizerFactory.createTokenizer();
+        tupleIter = new InvertedIndexTokenizingTupleIterator(tokenCmpFactories.length, btree.getFieldCount()
+                - tokenCmpFactories.length, tokenizer);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
new file mode 100644
index 0000000..6af3bd2
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
@@ -0,0 +1,240 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+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.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+
+public class InMemoryInvertedListCursor implements IInvertedListCursor {
+    private RangePredicate btreePred;
+    private BTreeAccessor btreeAccessor;
+    private IIndexCursor btreeCursor;
+    private boolean cursorNeedsClose = false;
+    private IIndexCursor countingCursor;
+    private MultiComparator tokenFieldsCmp;
+    private MultiComparator btreeCmp;
+    private final PermutingTupleReference resultTuple;
+    private final ConcatenatingTupleReference btreeSearchTuple;
+
+    private final ArrayTupleBuilder tokenTupleBuilder;
+    private final ArrayTupleReference tokenTuple = new ArrayTupleReference();
+
+    private int numElements = -1;
+
+    public InMemoryInvertedListCursor(int invListFieldCount, int tokenFieldCount) {
+        int[] fieldPermutation = new int[invListFieldCount];
+        for (int i = 0; i < invListFieldCount; i++) {
+            fieldPermutation[i] = tokenFieldCount + i;
+        }
+        resultTuple = new PermutingTupleReference(fieldPermutation);
+        // Concatenating the tuple with tokens, and the tuple with inverted-list elements.
+        btreeSearchTuple = new ConcatenatingTupleReference(2);
+        tokenTupleBuilder = new ArrayTupleBuilder(tokenFieldCount);
+    }
+
+    public void prepare(BTreeAccessor btreeAccessor, RangePredicate btreePred, MultiComparator tokenFieldsCmp,
+            MultiComparator btreeCmp) throws HyracksDataException, IndexException {
+        // Avoid object creation if this.btreeAccessor == btreeAccessor.
+        if (this.btreeAccessor != btreeAccessor) {
+            this.btreeAccessor = btreeAccessor;
+            this.btreeCursor = btreeAccessor.createSearchCursor();
+            this.countingCursor = btreeAccessor.createCountingSearchCursor();
+            this.btreePred = btreePred;
+            this.btreePred.setLowKeyComparator(tokenFieldsCmp);
+            this.btreePred.setHighKeyComparator(tokenFieldsCmp);
+            this.tokenFieldsCmp = tokenFieldsCmp;
+            this.btreeCmp = btreeCmp;
+        }
+    }
+
+    @Override
+    public int compareTo(IInvertedListCursor cursor) {
+        return size() - cursor.size();
+    }
+
+    public void reset(ITupleReference tuple) throws HyracksDataException, IndexException {
+        numElements = -1;
+        // Copy the tokens tuple for later use in btree probes.
+        TupleUtils.copyTuple(tokenTupleBuilder, tuple, tuple.getFieldCount());
+        tokenTuple.reset(tokenTupleBuilder.getFieldEndOffsets(), tokenTupleBuilder.getByteArray());
+        btreeSearchTuple.reset();
+        btreeSearchTuple.addTuple(tokenTuple);
+        btreeCursor.reset();
+        countingCursor.reset();
+    }
+
+    @Override
+    public void reset(int startPageId, int endPageId, int startOff, int numElements) {
+        // Do nothing
+    }
+
+    @Override
+    public void pinPages() throws HyracksDataException, IndexException {
+        btreePred.setLowKeyComparator(tokenFieldsCmp);
+        btreePred.setHighKeyComparator(tokenFieldsCmp);
+        btreePred.setLowKey(tokenTuple, true);
+        btreePred.setHighKey(tokenTuple, true);
+        btreeAccessor.search(btreeCursor, btreePred);
+        cursorNeedsClose = true;
+    }
+
+    @Override
+    public void unpinPages() throws HyracksDataException {
+        if (cursorNeedsClose) {
+            btreeCursor.close();
+            cursorNeedsClose = false;
+        }
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        return btreeCursor.hasNext();
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        btreeCursor.next();
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        resultTuple.reset(btreeCursor.getTuple());
+        return resultTuple;
+    }
+
+    @Override
+    public int size() {
+        if (numElements < 0) {
+            btreePred.setLowKeyComparator(tokenFieldsCmp);
+            btreePred.setHighKeyComparator(tokenFieldsCmp);
+            btreePred.setLowKey(tokenTuple, true);
+            btreePred.setHighKey(tokenTuple, true);
+
+            // Perform the count.
+            try {
+                btreeAccessor.search(countingCursor, btreePred);
+                while (countingCursor.hasNext()) {
+                    countingCursor.next();
+                    ITupleReference countTuple = countingCursor.getTuple();
+                    numElements = IntegerSerializerDeserializer.getInt(countTuple.getFieldData(0),
+                            countTuple.getFieldStart(0));
+                }
+            } catch (HyracksDataException e) {
+                e.printStackTrace();
+            } catch (IndexException e) {
+                e.printStackTrace();
+            } finally {
+                try {
+                    countingCursor.close();
+                } catch (HyracksDataException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+        return numElements;
+    }
+
+    @Override
+    public int getStartPageId() {
+        return 0;
+    }
+
+    @Override
+    public int getEndPageId() {
+        return 0;
+    }
+
+    @Override
+    public int getStartOff() {
+        return 0;
+    }
+
+    @Override
+    public boolean containsKey(ITupleReference searchTuple, MultiComparator invListCmp) throws HyracksDataException,
+            IndexException {
+        // Close cursor if necessary.
+        unpinPages();
+        btreeSearchTuple.addTuple(searchTuple);
+        btreePred.setLowKeyComparator(btreeCmp);
+        btreePred.setHighKeyComparator(btreeCmp);
+        btreePred.setLowKey(btreeSearchTuple, true);
+        btreePred.setHighKey(btreeSearchTuple, true);
+        try {
+            btreeAccessor.search(btreeCursor, btreePred);
+        } catch (TreeIndexException e) {
+            btreeSearchTuple.removeLastTuple();
+            throw new HyracksDataException(e);
+        }
+        boolean containsKey = false;
+        try {
+            containsKey = btreeCursor.hasNext();
+        } finally {
+            btreeCursor.close();
+            btreeCursor.reset();
+            btreeSearchTuple.removeLastTuple();
+        }
+        return containsKey;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    public String printInvList(ISerializerDeserializer[] serdes) throws HyracksDataException, IndexException {
+        StringBuilder strBuilder = new StringBuilder();
+        try {
+            while (btreeCursor.hasNext()) {
+                btreeCursor.next();
+                ITupleReference tuple = btreeCursor.getTuple();
+                ByteArrayInputStream inStream = new ByteArrayInputStream(tuple.getFieldData(1), tuple.getFieldStart(1),
+                        tuple.getFieldLength(1));
+                DataInput dataIn = new DataInputStream(inStream);
+                Object o = serdes[0].deserialize(dataIn);
+                strBuilder.append(o.toString() + " ");
+            }
+        } finally {
+            btreeCursor.close();
+            btreeCursor.reset();
+        }
+        try {
+            btreeAccessor.search(btreeCursor, btreePred);
+        } catch (TreeIndexException e) {
+            throw new HyracksDataException(e);
+        }
+        return strBuilder.toString();
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    public String printCurrentElement(ISerializerDeserializer[] serdes) throws HyracksDataException {
+        return null;
+    }
+}
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
new file mode 100644
index 0000000..7c3f4e4
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -0,0 +1,146 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+import java.util.ArrayList;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IPartitionedInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedListPartitions;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.PartitionedTOccurrenceSearcher;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.PartitionedInvertedIndexTokenizingTupleIterator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public class PartitionedInMemoryInvertedIndex extends InMemoryInvertedIndex implements IPartitionedInvertedIndex {
+
+    protected final ReentrantReadWriteLock partitionIndexLock = new ReentrantReadWriteLock(true);
+    protected short minPartitionIndex = Short.MAX_VALUE;
+    protected short maxPartitionIndex = Short.MIN_VALUE;
+
+    public PartitionedInMemoryInvertedIndex(IBufferCache memBufferCache, IFreePageManager memFreePageManager,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory) throws BTreeException {
+        super(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                tokenCmpFactories, tokenizerFactory);
+    }
+
+    @Override
+    public void insert(ITupleReference tuple, BTreeAccessor btreeAccessor, IIndexOperationContext ictx)
+            throws HyracksDataException, IndexException {
+        super.insert(tuple, btreeAccessor, ictx);
+        PartitionedInMemoryInvertedIndexOpContext ctx = (PartitionedInMemoryInvertedIndexOpContext) ictx;
+        PartitionedInvertedIndexTokenizingTupleIterator tupleIter = (PartitionedInvertedIndexTokenizingTupleIterator) ctx.tupleIter;
+        updatePartitionIndexes(tupleIter.getNumTokens());
+    }
+
+    @Override
+    public void clear() throws HyracksDataException {
+        super.clear();
+        minPartitionIndex = Short.MAX_VALUE;
+        maxPartitionIndex = Short.MIN_VALUE;
+    }
+
+    public void updatePartitionIndexes(short numTokens) {
+        partitionIndexLock.writeLock().lock();
+        if (numTokens < minPartitionIndex) {
+            minPartitionIndex = numTokens;
+        }
+        if (numTokens > maxPartitionIndex) {
+            maxPartitionIndex = numTokens;
+        }
+        partitionIndexLock.writeLock().unlock();
+    }
+
+    @Override
+    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new PartitionedInMemoryInvertedIndexAccessor(this, new PartitionedInMemoryInvertedIndexOpContext(btree,
+                tokenCmpFactories, tokenizerFactory));
+    }
+
+    @Override
+    public boolean openInvertedListPartitionCursors(IInvertedIndexSearcher searcher, IIndexOperationContext ictx,
+            short numTokensLowerBound, short numTokensUpperBound, InvertedListPartitions invListPartitions,
+            ArrayList<IInvertedListCursor> cursorsOrderedByTokens) throws HyracksDataException, IndexException {
+        short minPartitionIndex;
+        short maxPartitionIndex;
+        partitionIndexLock.readLock().lock();
+        minPartitionIndex = this.minPartitionIndex;
+        maxPartitionIndex = this.maxPartitionIndex;
+        partitionIndexLock.readLock().unlock();
+
+        if (minPartitionIndex == Short.MAX_VALUE && maxPartitionIndex == Short.MIN_VALUE) {
+            // Index must be empty.
+            return false;
+        }
+        short partitionStartIndex = minPartitionIndex;
+        short partitionEndIndex = maxPartitionIndex;
+        if (numTokensLowerBound >= 0) {
+            partitionStartIndex = (short) Math.max(minPartitionIndex, numTokensLowerBound);
+        }
+        if (numTokensUpperBound >= 0) {
+            partitionEndIndex = (short) Math.min(maxPartitionIndex, numTokensUpperBound);
+        }
+
+        PartitionedTOccurrenceSearcher partSearcher = (PartitionedTOccurrenceSearcher) searcher;
+        PartitionedInMemoryInvertedIndexOpContext ctx = (PartitionedInMemoryInvertedIndexOpContext) ictx;
+        ctx.setOperation(IndexOperation.SEARCH);
+        // We can pick either of the full low or high search key, since they should be identical here.
+        ITupleReference searchKey = partSearcher.getFullLowSearchKey();
+        ctx.btreePred.setLowKey(searchKey, true);
+        ctx.btreePred.setHighKey(searchKey, true);
+        // Go through all possibly partitions and see if the token matches.
+        // TODO: This procedure could be made more efficient by determining the next partition to search
+        // using the last existing partition and re-searching the BTree with an open interval as low key.
+        for (short i = partitionStartIndex; i <= partitionEndIndex; i++) {
+            partSearcher.setNumTokensBoundsInSearchKeys(i, i);
+            InMemoryInvertedListCursor inMemListCursor = (InMemoryInvertedListCursor) partSearcher
+                    .getCachedInvertedListCursor();
+            inMemListCursor.prepare(ctx.btreeAccessor, ctx.btreePred, ctx.tokenFieldsCmp, ctx.btreeCmp);
+            inMemListCursor.reset(searchKey);
+            invListPartitions.addInvertedListCursor(inMemListCursor, i);
+        }
+        return true;
+    }
+
+    @Override
+    public boolean isEmpty() {
+        partitionIndexLock.readLock().lock();
+        if (minPartitionIndex == Short.MAX_VALUE && maxPartitionIndex == Short.MIN_VALUE) {
+            // Index must be empty.
+            partitionIndexLock.readLock().unlock();
+            return true;
+        }
+        partitionIndexLock.readLock().unlock();
+        return false;
+    }
+}
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
new file mode 100644
index 0000000..813961c
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexAccessor.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2010 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.inmemory;
+
+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) {
+        super(index, opCtx);
+    }
+
+    protected IInvertedIndexSearcher createSearcher() {
+        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/inmemory/PartitionedInMemoryInvertedIndexOpContext.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java
new file mode 100644
index 0000000..f0e5046
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.PartitionedInvertedIndexTokenizingTupleIterator;
+
+public class PartitionedInMemoryInvertedIndexOpContext extends InMemoryInvertedIndexOpContext {
+
+    public PartitionedInMemoryInvertedIndexOpContext(BTree btree, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory) {
+        super(btree, tokenCmpFactories, tokenizerFactory);
+    }
+
+    protected void setTokenizingTupleIterator() {
+        IBinaryTokenizer tokenizer = tokenizerFactory.createTokenizer();
+        tupleIter = new PartitionedInvertedIndexTokenizingTupleIterator(tokenCmpFactories.length, btree.getFieldCount()
+                - tokenCmpFactories.length, tokenizer);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListBuilder.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListBuilder.java
new file mode 100644
index 0000000..fd12792
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListBuilder.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
+
+public class FixedSizeElementInvertedListBuilder implements IInvertedListBuilder {
+    private final int listElementSize;
+    private int listSize = 0;
+
+    private byte[] targetBuf;
+    private int pos;
+
+    public FixedSizeElementInvertedListBuilder(ITypeTraits[] invListFields) {
+        int tmp = 0;
+        for (int i = 0; i < invListFields.length; i++) {
+            tmp += invListFields[i].getFixedLength();
+        }
+        listElementSize = tmp;
+    }
+
+    @Override
+    public boolean startNewList(ITupleReference tuple, int tokenField) {
+        if (pos + listElementSize > targetBuf.length) {
+            return false;
+        } else {
+            listSize = 0;
+            return true;
+        }
+    }
+
+    @Override
+    public boolean appendElement(ITupleReference tuple, int numTokenFields, int numElementFields) {
+        if (pos + listElementSize > targetBuf.length) {
+            return false;
+        }
+
+        for (int i = 0; i < numElementFields; i++) {
+            int field = numTokenFields + i;
+            System.arraycopy(tuple.getFieldData(field), tuple.getFieldStart(field), targetBuf, pos,
+                    tuple.getFieldLength(field));
+        }
+
+        listSize++;
+        pos += listElementSize;
+
+        return true;
+    }
+
+    @Override
+    public void setTargetBuffer(byte[] targetBuf, int startPos) {
+        this.targetBuf = targetBuf;
+        this.pos = startPos;
+    }
+
+    @Override
+    public int getListSize() {
+        return listSize;
+    }
+
+    @Override
+    public int getPos() {
+        return pos;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListBuilderFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListBuilderFactory.java
new file mode 100644
index 0000000..ca2d3b6
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListBuilderFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2012 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.ondisk;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
+
+public class FixedSizeElementInvertedListBuilderFactory implements IInvertedListBuilderFactory {
+
+    private final ITypeTraits[] invListFields;
+
+    public FixedSizeElementInvertedListBuilderFactory(ITypeTraits[] invListFields) {
+        this.invListFields = invListFields;
+    }
+
+    @Override
+    public IInvertedListBuilder create() {
+        return new FixedSizeElementInvertedListBuilder(invListFields);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
new file mode 100644
index 0000000..f55a700
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
@@ -0,0 +1,295 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+
+public class FixedSizeElementInvertedListCursor implements IInvertedListCursor {
+
+    private final IBufferCache bufferCache;
+    private final int fileId;
+    private final int elementSize;
+    private int currentElementIx;
+    private int currentOff;
+    private int currentPageIx;
+
+    private int startPageId;
+    private int endPageId;
+    private int startOff;
+    private int numElements;
+
+    private final FixedSizeTupleReference tuple;
+    private ICachedPage[] pages = new ICachedPage[10];
+    private int[] elementIndexes = new int[10];
+    
+    private boolean pinned = false;
+
+    public FixedSizeElementInvertedListCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields) {
+        this.bufferCache = bufferCache;
+        this.fileId = fileId;
+        this.currentElementIx = 0;
+        this.currentPageIx = 0;
+
+        int tmp = 0;
+        for (int i = 0; i < invListFields.length; i++) {
+            tmp += invListFields[i].getFixedLength();
+        }
+        elementSize = tmp;
+        this.currentOff = -elementSize;
+        this.tuple = new FixedSizeTupleReference(invListFields);
+    }
+
+    @Override
+    public boolean hasNext() {
+        if (currentElementIx < numElements)
+            return true;
+        else
+            return false;
+    }
+
+    @Override
+    public void next() {
+        if (currentOff + 2 * elementSize > bufferCache.getPageSize()) {
+            currentPageIx++;
+            currentOff = 0;
+        } else {
+            currentOff += elementSize;
+        }
+        currentElementIx++;
+        tuple.reset(pages[currentPageIx].getBuffer().array(), currentOff);
+    }
+
+    @Override
+    public void pinPages() throws HyracksDataException {
+        if (pinned) {
+            return;
+        }
+        int pix = 0;
+        for (int i = startPageId; i <= endPageId; i++) {
+            pages[pix] = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i), false);
+            pages[pix].acquireReadLatch();
+            pix++;
+        }
+        pinned = true;
+    }
+
+    @Override
+    public void unpinPages() throws HyracksDataException {
+        int numPages = endPageId - startPageId + 1;
+        for (int i = 0; i < numPages; i++) {
+            pages[i].releaseReadLatch();
+            bufferCache.unpin(pages[i]);
+        }
+        pinned = false;
+    }
+
+    private void positionCursor(int elementIx) {
+        int numPages = endPageId - startPageId + 1;
+
+        currentPageIx = binarySearch(elementIndexes, 0, numPages, elementIx);
+        if (currentPageIx < 0) {
+            throw new IndexOutOfBoundsException("Requested index: " + elementIx + " from array with numElements: "
+                    + numElements);
+        }
+
+        if (currentPageIx == 0) {
+            currentOff = startOff + elementIx * elementSize;
+        } else {
+            int relativeElementIx = elementIx - elementIndexes[currentPageIx - 1] - 1;
+            currentOff = relativeElementIx * elementSize;
+        }
+
+        currentElementIx = elementIx;
+        tuple.reset(pages[currentPageIx].getBuffer().array(), currentOff);
+    }
+
+    @Override
+    public boolean containsKey(ITupleReference searchTuple, MultiComparator invListCmp) {
+        int mid;
+        int begin = 0;
+        int end = numElements - 1;
+
+        while (begin <= end) {
+            mid = (begin + end) / 2;
+            positionCursor(mid);
+            int cmp = invListCmp.compare(searchTuple, tuple);
+            if (cmp < 0) {
+                end = mid - 1;
+            } else if (cmp > 0) {
+                begin = mid + 1;
+            } else {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    @Override
+    public void reset(int startPageId, int endPageId, int startOff, int numElements) {
+        this.startPageId = startPageId;
+        this.endPageId = endPageId;
+        this.startOff = startOff;
+        this.numElements = numElements;
+        this.currentElementIx = 0;
+        this.currentPageIx = 0;
+        this.currentOff = startOff - elementSize;
+
+        int numPages = endPageId - startPageId + 1;
+        if (numPages > pages.length) {
+            pages = new ICachedPage[endPageId - startPageId + 1];
+            elementIndexes = new int[endPageId - startPageId + 1];
+        }
+
+        // fill elementIndexes
+        // first page
+        int cumulElements = (bufferCache.getPageSize() - startOff) / elementSize;
+        elementIndexes[0] = cumulElements - 1;
+
+        // middle, full pages
+        for (int i = 1; i < numPages - 1; i++) {
+            elementIndexes[i] = elementIndexes[i - 1] + (bufferCache.getPageSize() / elementSize);
+        }
+
+        // last page
+        elementIndexes[numPages - 1] = numElements - 1;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    public String printInvList(ISerializerDeserializer[] serdes) throws HyracksDataException {
+        int oldCurrentOff = currentOff;
+        int oldCurrentPageId = currentPageIx;
+        int oldCurrentElementIx = currentElementIx;
+
+        currentOff = startOff - elementSize;
+        currentPageIx = 0;
+        currentElementIx = 0;
+
+        StringBuilder strBuilder = new StringBuilder();
+
+        while (hasNext()) {
+            next();
+            for (int i = 0; i < tuple.getFieldCount(); i++) {
+                ByteArrayInputStream inStream = new ByteArrayInputStream(tuple.getFieldData(i), tuple.getFieldStart(i),
+                        tuple.getFieldLength(i));
+                DataInput dataIn = new DataInputStream(inStream);
+                Object o = serdes[i].deserialize(dataIn);
+                strBuilder.append(o.toString());
+                if (i + 1 < tuple.getFieldCount())
+                    strBuilder.append(",");
+            }
+            strBuilder.append(" ");
+        }
+
+        // reset previous state
+        currentOff = oldCurrentOff;
+        currentPageIx = oldCurrentPageId;
+        currentElementIx = oldCurrentElementIx;
+
+        return strBuilder.toString();
+    }
+
+    @SuppressWarnings("rawtypes")
+    public String printCurrentElement(ISerializerDeserializer[] serdes) throws HyracksDataException {
+        StringBuilder strBuilder = new StringBuilder();
+        for (int i = 0; i < tuple.getFieldCount(); i++) {
+            ByteArrayInputStream inStream = new ByteArrayInputStream(tuple.getFieldData(i), tuple.getFieldStart(i),
+                    tuple.getFieldLength(i));
+            DataInput dataIn = new DataInputStream(inStream);
+            Object o = serdes[i].deserialize(dataIn);
+            strBuilder.append(o.toString());
+            if (i + 1 < tuple.getFieldCount())
+                strBuilder.append(",");
+        }
+        return strBuilder.toString();
+    }
+
+    private int binarySearch(int[] arr, int arrStart, int arrLength, int key) {
+        int mid;
+        int begin = arrStart;
+        int end = arrStart + arrLength - 1;
+
+        while (begin <= end) {
+            mid = (begin + end) / 2;
+            int cmp = (key - arr[mid]);
+            if (cmp < 0) {
+                end = mid - 1;
+            } else if (cmp > 0) {
+                begin = mid + 1;
+            } else {
+                return mid;
+            }
+        }
+
+        if (begin > arr.length - 1)
+            return -1;
+        if (key < arr[begin])
+            return begin;
+        else
+            return -1;
+    }
+
+    @Override
+    public int compareTo(IInvertedListCursor invListCursor) {
+        return numElements - invListCursor.size();
+    }
+
+    @Override
+    public int getEndPageId() {
+        return endPageId;
+    }
+
+    @Override
+    public int size() {
+        return numElements;
+    }
+
+    @Override
+    public int getStartOff() {
+        return startOff;
+    }
+
+    @Override
+    public int getStartPageId() {
+        return startPageId;
+    }
+
+    public int getOffset() {
+        return currentOff;
+    }
+
+    public ICachedPage getPage() {
+        return pages[currentPageIx];
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return tuple;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java
new file mode 100644
index 0000000..d00c6d3
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+
+public class FixedSizeFrameTupleAccessor implements IFrameTupleAccessor {
+
+    private final int frameSize;
+    private ByteBuffer buffer;
+
+    private final ITypeTraits[] fields;
+    private final int[] fieldStartOffsets;
+    private final int tupleSize;
+
+    public FixedSizeFrameTupleAccessor(int frameSize, ITypeTraits[] fields) {
+        this.frameSize = frameSize;
+        this.fields = fields;
+        this.fieldStartOffsets = new int[fields.length];
+        this.fieldStartOffsets[0] = 0;
+        for (int i = 1; i < fields.length; i++) {
+            fieldStartOffsets[i] = fieldStartOffsets[i - 1] + fields[i - 1].getFixedLength();
+        }
+
+        int tmp = 0;
+        for (int i = 0; i < fields.length; i++) {
+            tmp += fields[i].getFixedLength();
+        }
+        tupleSize = tmp;
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return fields.length;
+    }
+
+    @Override
+    public int getFieldEndOffset(int tupleIndex, int fIdx) {
+        return getTupleStartOffset(tupleIndex) + fieldStartOffsets[fIdx] + fields[fIdx].getFixedLength();
+    }
+
+    @Override
+    public int getFieldLength(int tupleIndex, int fIdx) {
+        return fields[fIdx].getFixedLength();
+    }
+
+    @Override
+    public int getFieldSlotsLength() {
+        return 0;
+    }
+
+    @Override
+    public int getFieldStartOffset(int tupleIndex, int fIdx) {
+        return tupleIndex * tupleSize + fieldStartOffsets[fIdx];
+    }
+
+    @Override
+    public int getTupleCount() {
+        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
+    }
+
+    @Override
+    public int getTupleEndOffset(int tupleIndex) {
+        return getFieldEndOffset(tupleIndex, fields.length - 1);
+    }
+
+    @Override
+    public int getTupleStartOffset(int tupleIndex) {
+        return tupleIndex * tupleSize;
+    }
+
+    @Override
+    public void reset(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAppender.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAppender.java
new file mode 100644
index 0000000..2a8bf8d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAppender.java
@@ -0,0 +1,127 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+
+public class FixedSizeFrameTupleAppender {
+
+    private static final int TUPLE_COUNT_SIZE = 4;
+    private final int frameSize;
+    private final int tupleSize;
+    private ByteBuffer buffer;
+    private int tupleCount;
+    private int tupleDataEndOffset;
+
+    public FixedSizeFrameTupleAppender(int frameSize, ITypeTraits[] fields) {
+        this.frameSize = frameSize;
+        int tmp = 0;
+        for (int i = 0; i < fields.length; i++) {
+            tmp += fields[i].getFixedLength();
+        }
+        tupleSize = tmp;
+    }
+
+    public void reset(ByteBuffer buffer, boolean clear) {
+        this.buffer = buffer;
+        if (clear) {
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), 0);
+            tupleCount = 0;
+            tupleDataEndOffset = 0;
+        }
+    }
+
+    public boolean append(byte[] bytes, int offset) {
+        if (tupleDataEndOffset + tupleSize + TUPLE_COUNT_SIZE <= frameSize) {
+            System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset, tupleSize);
+            tupleDataEndOffset += tupleSize;
+            tupleCount++;
+            return true;
+        }
+        return false;
+    }
+
+    public boolean append(byte[] bytes, int offset, int length) {
+        if (tupleDataEndOffset + length + TUPLE_COUNT_SIZE <= frameSize) {
+            System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset, length);
+            tupleDataEndOffset += length;
+            return true;
+        }
+        return false;
+    }
+
+    public boolean append(int fieldValue) {
+        if (tupleDataEndOffset + 4 + TUPLE_COUNT_SIZE <= frameSize) {
+            buffer.putInt(tupleDataEndOffset, fieldValue);
+            tupleDataEndOffset += 4;
+            tupleCount++;
+            return true;
+        }
+        return false;
+    }
+
+    public boolean append(long fieldValue) {
+        if (tupleDataEndOffset + 8 + TUPLE_COUNT_SIZE <= frameSize) {
+            buffer.putLong(tupleDataEndOffset, fieldValue);
+            tupleDataEndOffset += 8;
+            tupleCount++;
+            return true;
+        }
+        return false;
+    }
+
+    public boolean append(char fieldValue) {
+        if (tupleDataEndOffset + 2 + TUPLE_COUNT_SIZE <= frameSize) {
+            buffer.putLong(tupleDataEndOffset, fieldValue);
+            tupleDataEndOffset += 2;
+            tupleCount++;
+            return true;
+        }
+        return false;
+    }
+
+    public boolean append(byte fieldValue) {
+        if (tupleDataEndOffset + 1 + TUPLE_COUNT_SIZE <= frameSize) {
+            buffer.put(tupleDataEndOffset, fieldValue);
+            tupleDataEndOffset += 1;
+            tupleCount++;
+            return true;
+        }
+        return false;
+    }
+
+    // returns true if an entire tuple fits
+    // returns false otherwise
+    public boolean hasSpace() {
+        return tupleDataEndOffset + tupleSize + TUPLE_COUNT_SIZE <= frameSize;
+    }
+
+    public void incrementTupleCount(int count) {
+        buffer.putInt(FrameHelper.getTupleCountOffset(frameSize),
+                buffer.getInt(FrameHelper.getTupleCountOffset(frameSize)) + count);
+    }
+
+    public int getTupleCount() {
+        return tupleCount;
+    }
+
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeTupleReference.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeTupleReference.java
new file mode 100644
index 0000000..97f6937
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeTupleReference.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class FixedSizeTupleReference implements ITupleReference {
+
+    private final ITypeTraits[] typeTraits;
+    private final int[] fieldStartOffsets;
+    private byte[] data;
+    private int startOff;
+
+    public FixedSizeTupleReference(ITypeTraits[] typeTraits) {
+        this.typeTraits = typeTraits;
+        this.fieldStartOffsets = new int[typeTraits.length];
+        this.fieldStartOffsets[0] = 0;
+        for (int i = 1; i < typeTraits.length; i++) {
+            fieldStartOffsets[i] = fieldStartOffsets[i - 1] + typeTraits[i - 1].getFixedLength();
+        }
+    }
+
+    public void reset(byte[] data, int startOff) {
+        this.data = data;
+        this.startOff = startOff;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return typeTraits.length;
+    }
+
+    @Override
+    public byte[] getFieldData(int fIdx) {
+        return data;
+    }
+
+    @Override
+    public int getFieldLength(int fIdx) {
+        return typeTraits[fIdx].getFixedLength();
+    }
+
+    @Override
+    public int getFieldStart(int fIdx) {
+        return startOff + fieldStartOffsets[fIdx];
+    }
+}
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
new file mode 100644
index 0000000..afeaf90
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -0,0 +1,663 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+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.data.std.primitive.IntegerPointable;
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.UnsortedInputException;
+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.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.InvertedIndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.TOccurrenceSearcher;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+/**
+ * An inverted index consists of two files: 1. a file storing (paginated)
+ * inverted lists 2. a BTree-file mapping from tokens to inverted lists.
+ * Implemented features: bulk loading and searching (based on T-Occurrence) Not
+ * implemented features: updates (insert/update/delete) Limitations: a query
+ * cannot exceed the size of a Hyracks frame.
+ */
+public class OnDiskInvertedIndex implements IInvertedIndex {
+    protected final IHyracksCommonContext ctx = new DefaultHyracksCommonContext();
+
+    // Schema of BTree tuples, set in constructor.    
+    protected final int invListStartPageIdField;
+    protected final int invListEndPageIdField;
+    protected final int invListStartOffField;
+    protected final int invListNumElementsField;
+
+    // Type traits to be appended to the token type trait which finally form the BTree field type traits.
+    protected static final ITypeTraits[] btreeValueTypeTraits = new ITypeTraits[4];
+    static {
+        // startPageId
+        btreeValueTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        // endPageId
+        btreeValueTypeTraits[1] = IntegerPointable.TYPE_TRAITS;
+        // startOff
+        btreeValueTypeTraits[2] = IntegerPointable.TYPE_TRAITS;
+        // numElements
+        btreeValueTypeTraits[3] = IntegerPointable.TYPE_TRAITS;
+    }
+
+    protected BTree btree;
+    protected int rootPageId = 0;
+    protected IBufferCache bufferCache;
+    protected IFileMapProvider fileMapProvider;
+    protected int fileId = -1;
+    protected final ITypeTraits[] invListTypeTraits;
+    protected final IBinaryComparatorFactory[] invListCmpFactories;
+    protected final ITypeTraits[] tokenTypeTraits;
+    protected final IBinaryComparatorFactory[] tokenCmpFactories;
+    protected final IInvertedListBuilder invListBuilder;
+    protected final int numTokenFields;
+    protected final int numInvListKeys;
+    protected final FileReference invListsFile;
+    // Last page id of inverted-lists file (inclusive). Set during bulk load.
+    protected int invListsMaxPageId = -1;
+    protected boolean isOpen = false;
+
+    public OnDiskInvertedIndex(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            IInvertedListBuilder invListBuilder, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, FileReference btreeFile, FileReference invListsFile)
+            throws IndexException {
+        this.bufferCache = bufferCache;
+        this.fileMapProvider = fileMapProvider;
+        this.invListBuilder = invListBuilder;
+        this.invListTypeTraits = invListTypeTraits;
+        this.invListCmpFactories = invListCmpFactories;
+        this.tokenTypeTraits = tokenTypeTraits;
+        this.tokenCmpFactories = tokenCmpFactories;
+        this.btree = BTreeUtils.createBTree(bufferCache, fileMapProvider, getBTreeTypeTraits(tokenTypeTraits),
+                tokenCmpFactories, BTreeLeafFrameType.REGULAR_NSM, btreeFile);
+        this.numTokenFields = btree.getComparatorFactories().length;
+        this.numInvListKeys = invListCmpFactories.length;
+        this.invListsFile = invListsFile;
+        this.invListStartPageIdField = numTokenFields;
+        this.invListEndPageIdField = numTokenFields + 1;
+        this.invListStartOffField = numTokenFields + 2;
+        this.invListNumElementsField = numTokenFields + 3;
+    }
+
+    @Override
+    public synchronized void create() throws HyracksDataException {
+        if (isOpen) {
+            throw new HyracksDataException("Failed to create since index is already open.");
+        }
+        btree.create();
+
+        boolean fileIsMapped = false;
+        synchronized (fileMapProvider) {
+            fileIsMapped = fileMapProvider.isMapped(invListsFile);
+            if (!fileIsMapped) {
+                bufferCache.createFile(invListsFile);
+            }
+            fileId = fileMapProvider.lookupFileId(invListsFile);
+            try {
+                // Also creates the file if it doesn't exist yet.
+                bufferCache.openFile(fileId);
+            } catch (HyracksDataException e) {
+                // Revert state of buffer cache since file failed to open.
+                if (!fileIsMapped) {
+                    bufferCache.deleteFile(fileId, false);
+                }
+                throw e;
+            }
+        }
+        bufferCache.closeFile(fileId);
+    }
+
+    @Override
+    public synchronized void activate() throws HyracksDataException {
+        if (isOpen) {
+            return;
+        }
+
+        btree.activate();
+        boolean fileIsMapped = false;
+        synchronized (fileMapProvider) {
+            fileIsMapped = fileMapProvider.isMapped(invListsFile);
+            if (!fileIsMapped) {
+                bufferCache.createFile(invListsFile);
+            }
+            fileId = fileMapProvider.lookupFileId(invListsFile);
+            try {
+                // Also creates the file if it doesn't exist yet.
+                bufferCache.openFile(fileId);
+            } catch (HyracksDataException e) {
+                // Revert state of buffer cache since file failed to open.
+                if (!fileIsMapped) {
+                    bufferCache.deleteFile(fileId, false);
+                }
+                throw e;
+            }
+        }
+
+        isOpen = true;
+    }
+
+    @Override
+    public synchronized void deactivate() throws HyracksDataException {
+        if (!isOpen) {
+            return;
+        }
+
+        btree.deactivate();
+        bufferCache.closeFile(fileId);
+
+        isOpen = false;
+    }
+
+    @Override
+    public synchronized void destroy() throws HyracksDataException {
+        if (isOpen) {
+            throw new HyracksDataException("Failed to destroy since index is already open.");
+        }
+
+        btree.destroy();
+        invListsFile.delete();
+        if (fileId == -1) {
+            return;
+        }
+
+        bufferCache.deleteFile(fileId, false);
+        fileId = -1;
+    }
+
+    @Override
+    public synchronized void clear() throws HyracksDataException {
+        if (!isOpen) {
+            throw new HyracksDataException("Failed to clear since index is not open.");
+        }
+        btree.clear();
+        bufferCache.closeFile(fileId);
+        bufferCache.deleteFile(fileId, false);
+        invListsFile.getFile().delete();
+
+        boolean fileIsMapped = false;
+        synchronized (fileMapProvider) {
+            fileIsMapped = fileMapProvider.isMapped(invListsFile);
+            if (!fileIsMapped) {
+                bufferCache.createFile(invListsFile);
+            }
+            fileId = fileMapProvider.lookupFileId(invListsFile);
+            try {
+                // Also creates the file if it doesn't exist yet.
+                bufferCache.openFile(fileId);
+            } catch (HyracksDataException e) {
+                // Revert state of buffer cache since file failed to open.
+                if (!fileIsMapped) {
+                    bufferCache.deleteFile(fileId, false);
+                }
+                throw e;
+            }
+        }
+    }
+
+    @Override
+    public IInvertedListCursor createInvertedListCursor() {
+        return new FixedSizeElementInvertedListCursor(bufferCache, fileId, invListTypeTraits);
+    }
+
+    @Override
+    public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey,
+            IIndexOperationContext ictx) throws HyracksDataException, IndexException {
+        OnDiskInvertedIndexOpContext ctx = (OnDiskInvertedIndexOpContext) ictx;
+        ctx.btreePred.setLowKeyComparator(ctx.searchCmp);
+        ctx.btreePred.setHighKeyComparator(ctx.searchCmp);
+        ctx.btreePred.setLowKey(searchKey, true);
+        ctx.btreePred.setHighKey(searchKey, true);
+        ctx.btreeAccessor.search(ctx.btreeCursor, ctx.btreePred);
+        try {
+            if (ctx.btreeCursor.hasNext()) {
+                ctx.btreeCursor.next();
+                resetInvertedListCursor(ctx.btreeCursor.getTuple(), listCursor);
+            } else {
+                listCursor.reset(0, 0, 0, 0);
+            }
+        } finally {
+            ctx.btreeCursor.close();
+            ctx.btreeCursor.reset();
+        }
+    }
+
+    public void resetInvertedListCursor(ITupleReference btreeTuple, IInvertedListCursor listCursor) {
+        int startPageId = IntegerSerializerDeserializer.getInt(btreeTuple.getFieldData(invListStartPageIdField),
+                btreeTuple.getFieldStart(invListStartPageIdField));
+        int endPageId = IntegerSerializerDeserializer.getInt(btreeTuple.getFieldData(invListEndPageIdField),
+                btreeTuple.getFieldStart(invListEndPageIdField));
+        int startOff = IntegerSerializerDeserializer.getInt(btreeTuple.getFieldData(invListStartOffField),
+                btreeTuple.getFieldStart(invListStartOffField));
+        int numElements = IntegerSerializerDeserializer.getInt(btreeTuple.getFieldData(invListNumElementsField),
+                btreeTuple.getFieldStart(invListNumElementsField));
+        listCursor.reset(startPageId, endPageId, startOff, numElements);
+    }
+
+    public final class OnDiskInvertedIndexBulkLoader implements IIndexBulkLoader {
+        private final ArrayTupleBuilder btreeTupleBuilder;
+        private final ArrayTupleReference btreeTupleReference;
+        private final IIndexBulkLoader btreeBulkloader;
+
+        private int currentInvListStartPageId;
+        private int currentInvListStartOffset;
+        private final ArrayTupleBuilder lastTupleBuilder;
+        private final ArrayTupleReference lastTuple;
+
+        private int currentPageId;
+        private ICachedPage currentPage;
+        private final MultiComparator tokenCmp;
+        private final MultiComparator invListCmp;
+
+        private final boolean verifyInput;
+        private final MultiComparator allCmp;
+
+        public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+                int startPageId, int fileId) throws IndexException, HyracksDataException {
+            this.verifyInput = verifyInput;
+            this.tokenCmp = MultiComparator.create(btree.getComparatorFactories());
+            this.invListCmp = MultiComparator.create(invListCmpFactories);
+            if (verifyInput) {
+                allCmp = MultiComparator.create(btree.getComparatorFactories(), invListCmpFactories);
+            } else {
+                allCmp = null;
+            }
+            this.btreeTupleBuilder = new ArrayTupleBuilder(btree.getFieldCount());
+            this.btreeTupleReference = new ArrayTupleReference();
+            this.lastTupleBuilder = new ArrayTupleBuilder(numTokenFields + numInvListKeys);
+            this.lastTuple = new ArrayTupleReference();
+            this.btreeBulkloader = btree.createBulkLoader(btreeFillFactor, verifyInput, numElementsHint);
+            currentPageId = startPageId;
+            currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
+            currentPage.acquireWriteLatch();
+            invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
+        }
+
+        public void pinNextPage() throws HyracksDataException {
+            currentPage.releaseWriteLatch();
+            bufferCache.unpin(currentPage);
+            currentPageId++;
+            currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
+            currentPage.acquireWriteLatch();
+        }
+
+        private void createAndInsertBTreeTuple() throws IndexException, HyracksDataException {
+            // Build tuple.        
+            btreeTupleBuilder.reset();
+            DataOutput output = btreeTupleBuilder.getDataOutput();
+            // Add key fields.
+            for (int i = 0; i < numTokenFields; i++) {
+                btreeTupleBuilder.addField(lastTuple.getFieldData(i), lastTuple.getFieldStart(i),
+                        lastTuple.getFieldLength(i));
+            }
+            // Add inverted-list 'pointer' value fields.
+            try {
+                output.writeInt(currentInvListStartPageId);
+                btreeTupleBuilder.addFieldEndOffset();
+                output.writeInt(currentPageId);
+                btreeTupleBuilder.addFieldEndOffset();
+                output.writeInt(currentInvListStartOffset);
+                btreeTupleBuilder.addFieldEndOffset();
+                output.writeInt(invListBuilder.getListSize());
+                btreeTupleBuilder.addFieldEndOffset();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+            // Reset tuple reference and add it into the BTree load.
+            btreeTupleReference.reset(btreeTupleBuilder.getFieldEndOffsets(), btreeTupleBuilder.getByteArray());
+            btreeBulkloader.add(btreeTupleReference);
+        }
+
+        /**
+         * Assumptions:
+         * The first btree.getMultiComparator().getKeyFieldCount() fields in tuple
+         * are btree keys (e.g., a string token).
+         * The next invListCmp.getKeyFieldCount() fields in tuple are keys of the
+         * inverted list (e.g., primary key).
+         * Key fields of inverted list are fixed size.
+         */
+        @Override
+        public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
+            boolean firstElement = lastTupleBuilder.getSize() == 0;
+            boolean startNewList = firstElement;
+            if (!firstElement) {
+                // If the current and the last token don't match, we start a new list.
+                lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
+                startNewList = tokenCmp.compare(tuple, lastTuple) != 0;
+            }
+            if (startNewList) {
+                if (!firstElement) {
+                    // Create entry in btree for last inverted list.
+                    createAndInsertBTreeTuple();
+                }
+                if (!invListBuilder.startNewList(tuple, numTokenFields)) {
+                    pinNextPage();
+                    invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
+                    if (!invListBuilder.startNewList(tuple, numTokenFields)) {
+                        throw new IllegalStateException("Failed to create first inverted list.");
+                    }
+                }
+                currentInvListStartPageId = currentPageId;
+                currentInvListStartOffset = invListBuilder.getPos();
+            } else {
+                if (invListCmp.compare(tuple, lastTuple, numTokenFields) == 0) {
+                    // Duplicate inverted-list element.
+                    return;
+                }
+            }
+
+            // Append to current inverted list.
+            if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
+                pinNextPage();
+                invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
+                if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
+                    throw new IllegalStateException(
+                            "Failed to append element to inverted list after switching to a new page.");
+                }
+            }
+
+            if (verifyInput && lastTupleBuilder.getSize() != 0) {
+                if (allCmp.compare(tuple, lastTuple) <= 0) {
+                    throw new UnsortedInputException(
+                            "Input stream given to OnDiskInvertedIndex bulk load is not sorted.");
+                }
+            }
+
+            // Remember last tuple by creating a copy.
+            // TODO: This portion can be optimized by only copying the token when it changes, and using the last appended inverted-list element as a reference.
+            lastTupleBuilder.reset();
+            for (int i = 0; i < tuple.getFieldCount(); i++) {
+                lastTupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+            }
+        }
+
+        @Override
+        public void end() throws IndexException, HyracksDataException {
+            // The last tuple builder is empty if add() was never called.
+            if (lastTupleBuilder.getSize() != 0) {
+                createAndInsertBTreeTuple();
+            }
+            btreeBulkloader.end();
+
+            if (currentPage != null) {
+                currentPage.releaseWriteLatch();
+                bufferCache.unpin(currentPage);
+            }
+            invListsMaxPageId = currentPageId;
+        }
+    }
+
+    @Override
+    public IBufferCache getBufferCache() {
+        return bufferCache;
+    }
+
+    public int getInvListsFileId() {
+        return fileId;
+    }
+
+    public int getInvListsMaxPageId() {
+        return invListsMaxPageId;
+    }
+
+    public IBinaryComparatorFactory[] getInvListCmpFactories() {
+        return invListCmpFactories;
+    }
+
+    public ITypeTraits[] getInvListTypeTraits() {
+        return invListTypeTraits;
+    }
+
+    public BTree getBTree() {
+        return btree;
+    }
+
+    public class OnDiskInvertedIndexAccessor implements IInvertedIndexAccessor {
+        private final OnDiskInvertedIndex index;
+        private final IInvertedIndexSearcher searcher;
+        private final IIndexOperationContext opCtx = new OnDiskInvertedIndexOpContext(btree);
+
+        public OnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) {
+            this.index = index;
+            this.searcher = new TOccurrenceSearcher(ctx, index);
+        }
+
+        // Let subclasses initialize.
+        protected OnDiskInvertedIndexAccessor(OnDiskInvertedIndex index, IInvertedIndexSearcher searcher) {
+            this.index = index;
+            this.searcher = searcher;
+        }
+
+        @Override
+        public IIndexCursor createSearchCursor() {
+            return new OnDiskInvertedIndexSearchCursor(searcher, index.getInvListTypeTraits().length);
+        }
+
+        @Override
+        public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
+                IndexException {
+            searcher.search((OnDiskInvertedIndexSearchCursor) cursor, (InvertedIndexSearchPredicate) searchPred, opCtx);
+        }
+
+        @Override
+        public IInvertedListCursor createInvertedListCursor() {
+            return index.createInvertedListCursor();
+        }
+
+        @Override
+        public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference searchKey)
+                throws HyracksDataException, IndexException {
+            index.openInvertedListCursor(listCursor, searchKey, opCtx);
+        }
+
+        @Override
+        public IIndexCursor createRangeSearchCursor() {
+            return new OnDiskInvertedIndexRangeSearchCursor(index, opCtx);
+        }
+
+        @Override
+        public void rangeSearch(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
+                IndexException {
+            OnDiskInvertedIndexRangeSearchCursor rangeSearchCursor = (OnDiskInvertedIndexRangeSearchCursor) cursor;
+            rangeSearchCursor.open(null, searchPred);
+        }
+
+        @Override
+        public void insert(ITupleReference tuple) throws HyracksDataException, IndexException {
+            throw new UnsupportedOperationException("Insert not supported by inverted index.");
+        }
+
+        @Override
+        public void update(ITupleReference tuple) throws HyracksDataException, IndexException {
+            throw new UnsupportedOperationException("Update not supported by inverted index.");
+        }
+
+        @Override
+        public void delete(ITupleReference tuple) throws HyracksDataException, IndexException {
+            throw new UnsupportedOperationException("Delete not supported by inverted index.");
+        }
+
+        @Override
+        public void upsert(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+            throw new UnsupportedOperationException("Upsert not supported by inverted index.");
+        }
+    }
+
+    @Override
+    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new OnDiskInvertedIndexAccessor(this);
+    }
+
+    // This is just a dummy hyracks context for allocating frames for temporary
+    // results during inverted index searches.
+    // TODO: In the future we should use the real HyracksTaskContext to track
+    // frame usage.
+    public static class DefaultHyracksCommonContext implements IHyracksCommonContext {
+        private final int FRAME_SIZE = 32768;
+
+        @Override
+        public int getFrameSize() {
+            return FRAME_SIZE;
+        }
+
+        @Override
+        public IIOManager getIOManager() {
+            return null;
+        }
+
+        @Override
+        public ByteBuffer allocateFrame() {
+            return ByteBuffer.allocate(FRAME_SIZE);
+        }
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+            throws IndexException {
+        try {
+            return new OnDiskInvertedIndexBulkLoader(fillFactor, verifyInput, numElementsHint, rootPageId, fileId);
+        } catch (HyracksDataException e) {
+            throw new InvertedIndexException(e);
+        }
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        btree.validate();
+        // Scan the btree and validate the order of elements in each inverted-list.
+        IIndexAccessor btreeAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+        IIndexCursor btreeCursor = btreeAccessor.createSearchCursor();
+        MultiComparator btreeCmp = MultiComparator.createIgnoreFieldLength(btree.getComparatorFactories());
+        RangePredicate rangePred = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);
+        int[] fieldPermutation = new int[tokenTypeTraits.length];
+        for (int i = 0; i < tokenTypeTraits.length; i++) {
+            fieldPermutation[i] = i;
+        }
+        PermutingTupleReference tokenTuple = new PermutingTupleReference(fieldPermutation);
+
+        IInvertedIndexAccessor invIndexAccessor = (IInvertedIndexAccessor) createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        IInvertedListCursor invListCursor = invIndexAccessor.createInvertedListCursor();
+        MultiComparator invListCmp = MultiComparator.createIgnoreFieldLength(invListCmpFactories);
+
+        try {
+            // Search key for finding an inverted-list in the actual index.
+            ArrayTupleBuilder prevBuilder = new ArrayTupleBuilder(invListTypeTraits.length);
+            ArrayTupleReference prevTuple = new ArrayTupleReference();
+            btreeAccessor.search(btreeCursor, rangePred);
+            while (btreeCursor.hasNext()) {
+                btreeCursor.next();
+                tokenTuple.reset(btreeCursor.getTuple());
+                // Validate inverted list by checking that the elements are totally ordered.
+                invIndexAccessor.openInvertedListCursor(invListCursor, tokenTuple);
+                invListCursor.pinPages();
+                try {
+                    if (invListCursor.hasNext()) {
+                        invListCursor.next();
+                        ITupleReference invListElement = invListCursor.getTuple();
+                        // Initialize prev tuple.
+                        TupleUtils.copyTuple(prevBuilder, invListElement, invListElement.getFieldCount());
+                        prevTuple.reset(prevBuilder.getFieldEndOffsets(), prevBuilder.getByteArray());
+                    }
+                    while (invListCursor.hasNext()) {
+                        invListCursor.next();
+                        ITupleReference invListElement = invListCursor.getTuple();
+                        // Compare with previous element.
+                        if (invListCmp.compare(invListElement, prevTuple) <= 0) {
+                            throw new HyracksDataException("Index validation failed.");
+                        }
+                        // Set new prevTuple.
+                        TupleUtils.copyTuple(prevBuilder, invListElement, invListElement.getFieldCount());
+                        prevTuple.reset(prevBuilder.getFieldEndOffsets(), prevBuilder.getByteArray());
+                    }
+                } finally {
+                    invListCursor.unpinPages();
+                }
+            }
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        } finally {
+            btreeCursor.close();
+        }
+    }
+
+    @Override
+    public long getMemoryAllocationSize() {
+        return 0;
+    }
+
+    protected static ITypeTraits[] getBTreeTypeTraits(ITypeTraits[] tokenTypeTraits) {
+        ITypeTraits[] btreeTypeTraits = new ITypeTraits[tokenTypeTraits.length + btreeValueTypeTraits.length];
+        // Set key type traits.
+        for (int i = 0; i < tokenTypeTraits.length; i++) {
+            btreeTypeTraits[i] = tokenTypeTraits[i];
+        }
+        // Set value type traits.
+        for (int i = 0; i < btreeValueTypeTraits.length; i++) {
+            btreeTypeTraits[i + tokenTypeTraits.length] = btreeValueTypeTraits[i];
+        }
+        return btreeTypeTraits;
+    }
+
+    @Override
+    public ITypeTraits[] getTokenTypeTraits() {
+        return tokenTypeTraits;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getTokenCmpFactories() {
+        return tokenCmpFactories;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
new file mode 100644
index 0000000..5401519
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2012 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.ondisk;
+
+import java.io.File;
+
+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.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.IndexFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class OnDiskInvertedIndexFactory extends IndexFactory<IInvertedIndex> {
+
+    protected final IInvertedListBuilderFactory invListBuilderFactory;
+    protected final ITypeTraits[] invListTypeTraits;
+    protected final IBinaryComparatorFactory[] invListCmpFactories;
+    protected final ITypeTraits[] tokenTypeTraits;
+    protected final IBinaryComparatorFactory[] tokenCmpFactories;
+    protected final IInvertedIndexFileNameMapper fileNameMapper;
+
+    public OnDiskInvertedIndexFactory(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            IInvertedListBuilderFactory invListBuilderFactory, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IInvertedIndexFileNameMapper fileNameMapper) {
+        super(bufferCache, fileMapProvider, null);
+        this.invListBuilderFactory = invListBuilderFactory;
+        this.invListTypeTraits = invListTypeTraits;
+        this.invListCmpFactories = invListCmpFactories;
+        this.tokenTypeTraits = tokenTypeTraits;
+        this.tokenCmpFactories = tokenCmpFactories;
+        this.fileNameMapper = fileNameMapper;
+    }
+
+    @Override
+    public IInvertedIndex createIndexInstance(FileReference dictBTreeFile) throws IndexException {
+        String invListsFilePath = fileNameMapper.getInvListsFilePath(dictBTreeFile.getFile().getPath());
+        FileReference invListsFile = new FileReference(new File(invListsFilePath));
+        IInvertedListBuilder invListBuilder = invListBuilderFactory.create();
+        return new OnDiskInvertedIndex(bufferCache, fileMapProvider, invListBuilder, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
new file mode 100644
index 0000000..9e6194e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+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.IIndexOperationContext;
+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;
+
+public class OnDiskInvertedIndexOpContext implements IIndexOperationContext {
+
+    public final RangePredicate btreePred = new RangePredicate(null, null, true, true, null, null);
+    public IIndexAccessor btreeAccessor;
+    public IIndexCursor btreeCursor;
+    public MultiComparator searchCmp;
+    // For prefix search on partitioned indexes.
+    public MultiComparator prefixSearchCmp;
+
+    public OnDiskInvertedIndexOpContext(BTree btree) {
+        // TODO: Ignore opcallbacks for now.
+        btreeAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        btreeCursor = btreeAccessor.createSearchCursor();
+        searchCmp = MultiComparator.createIgnoreFieldLength(btree.getComparatorFactories());
+        if (btree.getComparatorFactories().length > 1) {
+            prefixSearchCmp = MultiComparator.create(btree.getComparatorFactories(), 0, 1);
+        }
+    }
+
+    @Override
+    public void reset() {
+        // Nothing to be done here, only search operation supported.
+    }
+
+    @Override
+    public void setOperation(IndexOperation newOp) {
+        // Nothing to be done here, only search operation supported.
+    }
+
+    @Override
+    public IndexOperation getOperation() {
+        return IndexOperation.SEARCH;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
new file mode 100644
index 0000000..b41b0d1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2009-2012 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.ondisk;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+
+/**
+ * Scans a range of tokens, returning tuples containing a token and an inverted-list element.
+ */
+public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor {
+
+    private final BTree btree;
+    private final IIndexAccessor btreeAccessor;
+    private final IInvertedIndex invIndex;
+    private final IIndexOperationContext opCtx;
+    private final IInvertedListCursor invListCursor;
+    private boolean unpinNeeded;
+    
+    private final IIndexCursor btreeCursor;
+    private RangePredicate btreePred;
+
+    private final PermutingTupleReference tokenTuple;
+    private ConcatenatingTupleReference concatTuple;
+
+    public OnDiskInvertedIndexRangeSearchCursor(IInvertedIndex invIndex, IIndexOperationContext opCtx) {
+        this.btree = ((OnDiskInvertedIndex) invIndex).getBTree();
+        this.btreeAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        this.invIndex = invIndex;
+        this.opCtx = opCtx;
+        // Project away non-token fields of the BTree tuples.
+        int[] fieldPermutation = new int[invIndex.getTokenTypeTraits().length];
+        for (int i = 0; i < invIndex.getTokenTypeTraits().length; i++) {
+            fieldPermutation[i] = i;
+        }
+        tokenTuple = new PermutingTupleReference(fieldPermutation);
+        btreeCursor = btreeAccessor.createSearchCursor();
+        concatTuple = new ConcatenatingTupleReference(2);
+        invListCursor = invIndex.createInvertedListCursor();
+        unpinNeeded = false;
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException, IndexException {
+        this.btreePred = (RangePredicate) searchPred;
+        try {
+            btreeAccessor.search(btreeCursor, btreePred);
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }        
+        invListCursor.pinPages();
+        unpinNeeded = true;
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        if (invListCursor.hasNext()) {
+            return true;
+        }
+        if (unpinNeeded) {
+            invListCursor.unpinPages();
+            unpinNeeded = false;
+        }
+        if (!btreeCursor.hasNext()) {
+            return false;
+        }
+        btreeCursor.next();
+        tokenTuple.reset(btreeCursor.getTuple());
+        try {
+            invIndex.openInvertedListCursor(invListCursor, tokenTuple, opCtx);
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+        invListCursor.pinPages();
+        invListCursor.hasNext();
+        unpinNeeded = true;
+        concatTuple.reset();
+        concatTuple.addTuple(tokenTuple);
+        return true;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        invListCursor.next();
+        if (concatTuple.hasMaxTuples()) {
+            concatTuple.removeLastTuple();
+        }
+        concatTuple.addTuple(invListCursor.getTuple());
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (unpinNeeded) {
+            invListCursor.unpinPages();
+            unpinNeeded = false;
+        }
+        btreeCursor.close();
+    }
+
+    @Override
+    public void reset() throws HyracksDataException, IndexException {
+        if (unpinNeeded) {
+            invListCursor.unpinPages();
+            unpinNeeded = false;
+        }
+        btreeCursor.close();
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return concatTuple;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java
new file mode 100644
index 0000000..3060ef4
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
+
+public class OnDiskInvertedIndexSearchCursor implements IIndexCursor {
+
+    private List<ByteBuffer> resultBuffers;
+    private int numResultBuffers;
+    private int currentBufferIndex = 0;
+    private int tupleIndex = 0;
+    private final IInvertedIndexSearcher invIndexSearcher;
+    private final IFrameTupleAccessor fta;
+    private final FixedSizeTupleReference frameTuple;
+    private final PermutingTupleReference resultTuple;
+    
+    public OnDiskInvertedIndexSearchCursor(IInvertedIndexSearcher invIndexSearcher, int numInvListFields) {
+        this.invIndexSearcher = invIndexSearcher;
+        this.fta = invIndexSearcher.createResultFrameTupleAccessor();
+        this.frameTuple = (FixedSizeTupleReference) invIndexSearcher.createResultFrameTupleReference();
+        // Project away the occurrence count from the result tuples.
+        int[] fieldPermutation = new int[numInvListFields];
+        for (int i = 0; i < numInvListFields; i++) {
+            fieldPermutation[i] = i;
+        }
+        resultTuple = new PermutingTupleReference(fieldPermutation);
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        currentBufferIndex = 0;
+        tupleIndex = 0;
+        resultBuffers = invIndexSearcher.getResultBuffers();
+        numResultBuffers = invIndexSearcher.getNumValidResultBuffers();
+        if (numResultBuffers > 0) {
+            fta.reset(resultBuffers.get(0));
+        }
+    }
+    
+    @Override
+    public boolean hasNext() {
+        if (currentBufferIndex < numResultBuffers && tupleIndex < fta.getTupleCount()) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public void next() {
+        frameTuple.reset(fta.getBuffer().array(), fta.getTupleStartOffset(tupleIndex));
+        resultTuple.reset(frameTuple);
+        tupleIndex++;
+        if (tupleIndex >= fta.getTupleCount()) {
+            if (currentBufferIndex + 1 < numResultBuffers) {
+                currentBufferIndex++;
+                fta.reset(resultBuffers.get(currentBufferIndex));
+                tupleIndex = 0;
+            }
+        }        
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return resultTuple;
+    }
+
+    @Override
+    public void reset() {
+        currentBufferIndex = 0;
+        tupleIndex = 0;
+        invIndexSearcher.reset();
+        resultBuffers = invIndexSearcher.getResultBuffers();
+        numResultBuffers = invIndexSearcher.getNumValidResultBuffers();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        currentBufferIndex = 0;
+        tupleIndex = 0;
+        resultBuffers = null;
+        numResultBuffers = 0;
+    }
+}
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
new file mode 100644
index 0000000..6e395e7
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -0,0 +1,116 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+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.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IPartitionedInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedListPartitions;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.PartitionedTOccurrenceSearcher;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class PartitionedOnDiskInvertedIndex extends OnDiskInvertedIndex implements IPartitionedInvertedIndex {
+
+    protected final int PARTITIONING_NUM_TOKENS_FIELD = 1;
+
+    public PartitionedOnDiskInvertedIndex(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            IInvertedListBuilder invListBuilder, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, FileReference btreeFile, FileReference invListsFile)
+            throws IndexException {
+        super(bufferCache, fileMapProvider, invListBuilder, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                tokenCmpFactories, btreeFile, invListsFile);
+    }
+
+    public class PartitionedOnDiskInvertedIndexAccessor extends OnDiskInvertedIndexAccessor {
+        public PartitionedOnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) {
+            super(index, new PartitionedTOccurrenceSearcher(ctx, index));
+        }
+    }
+
+    @Override
+    public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new PartitionedOnDiskInvertedIndexAccessor(this);
+    }
+
+    @Override
+    public boolean openInvertedListPartitionCursors(IInvertedIndexSearcher searcher, IIndexOperationContext ictx,
+            short numTokensLowerBound, short numTokensUpperBound, InvertedListPartitions invListPartitions,
+            ArrayList<IInvertedListCursor> cursorsOrderedByTokens) throws HyracksDataException, IndexException {
+        PartitionedTOccurrenceSearcher partSearcher = (PartitionedTOccurrenceSearcher) searcher;
+        OnDiskInvertedIndexOpContext ctx = (OnDiskInvertedIndexOpContext) ictx;
+        ITupleReference lowSearchKey = null;
+        ITupleReference highSearchKey = null;
+        partSearcher.setNumTokensBoundsInSearchKeys(numTokensLowerBound, numTokensUpperBound);
+        if (numTokensLowerBound < 0) {
+            ctx.btreePred.setLowKeyComparator(ctx.prefixSearchCmp);
+            lowSearchKey = partSearcher.getPrefixSearchKey();
+        } else {
+            ctx.btreePred.setLowKeyComparator(ctx.searchCmp);
+            lowSearchKey = partSearcher.getFullLowSearchKey();
+        }
+        if (numTokensUpperBound < 0) {
+            ctx.btreePred.setHighKeyComparator(ctx.prefixSearchCmp);
+            highSearchKey = partSearcher.getPrefixSearchKey();
+        } else {
+            ctx.btreePred.setHighKeyComparator(ctx.searchCmp);
+            highSearchKey = partSearcher.getFullHighSearchKey();
+        }
+        ctx.btreePred.setLowKey(lowSearchKey, true);
+        ctx.btreePred.setHighKey(highSearchKey, true);
+        ctx.btreeAccessor.search(ctx.btreeCursor, ctx.btreePred);
+        boolean tokenExists = false;
+        try {
+            while (ctx.btreeCursor.hasNext()) {
+                ctx.btreeCursor.next();
+                ITupleReference btreeTuple = ctx.btreeCursor.getTuple();
+                short numTokens = ShortSerializerDeserializer.getShort(
+                        btreeTuple.getFieldData(PARTITIONING_NUM_TOKENS_FIELD),
+                        btreeTuple.getFieldStart(PARTITIONING_NUM_TOKENS_FIELD));
+                IInvertedListCursor invListCursor = partSearcher.getCachedInvertedListCursor();
+                resetInvertedListCursor(btreeTuple, invListCursor);
+                cursorsOrderedByTokens.add(invListCursor);
+                invListPartitions.addInvertedListCursor(invListCursor, numTokens);
+                tokenExists = true;
+            }
+        } finally {
+            ctx.btreeCursor.close();
+            ctx.btreeCursor.reset();
+        }
+        return tokenExists;
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return false;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
new file mode 100644
index 0000000..854a30f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2012 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.ondisk;
+
+import java.io.File;
+
+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.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class PartitionedOnDiskInvertedIndexFactory extends OnDiskInvertedIndexFactory {
+    
+    public PartitionedOnDiskInvertedIndexFactory(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            IInvertedListBuilderFactory invListBuilderFactory, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IInvertedIndexFileNameMapper fileNameMapper) {
+        super(bufferCache, fileMapProvider, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                tokenCmpFactories, fileNameMapper);
+    }
+
+    @Override
+    public IInvertedIndex createIndexInstance(FileReference dictBTreeFile) throws IndexException {
+        String invListsFilePath = fileNameMapper.getInvListsFilePath(dictBTreeFile.getFile().getPath());
+        FileReference invListsFile = new FileReference(new File(invListsFilePath));
+        IInvertedListBuilder invListBuilder = invListBuilderFactory.create();
+        return new PartitionedOnDiskInvertedIndex(bufferCache, fileMapProvider, invListBuilder, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile);
+    }
+}
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
new file mode 100644
index 0000000..d973967
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -0,0 +1,154 @@
+/*
+ * Copyright 2009-2012 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.search;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+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.comm.io.FrameTupleAccessor;
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IObjectFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeFrameTupleAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.ObjectCache;
+
+public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearcher {
+    protected static final RecordDescriptor QUERY_TOKEN_REC_DESC = new RecordDescriptor(
+            new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
+
+    protected final int OBJECT_CACHE_INIT_SIZE = 10;
+    protected final int OBJECT_CACHE_EXPAND_SIZE = 10;
+
+    protected final IHyracksCommonContext ctx;
+
+    protected final InvertedListMerger invListMerger;
+    protected final SearchResult searchResult;
+    protected final IInvertedIndex invIndex;
+    protected final MultiComparator invListCmp;
+
+    protected final ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(QUERY_TOKEN_REC_DESC.getFieldCount());
+    protected final ByteBuffer queryTokenFrame;
+    protected final FrameTupleAppender queryTokenAppender;
+    protected final FrameTupleAccessor queryTokenAccessor;
+    protected final FrameTupleReference searchKey = new FrameTupleReference();
+
+    protected int occurrenceThreshold;
+
+    protected final IObjectFactory<IInvertedListCursor> invListCursorFactory;
+    protected final ObjectCache<IInvertedListCursor> invListCursorCache;
+
+    public AbstractTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+        this.ctx = ctx;
+        this.invListMerger = new InvertedListMerger(ctx, invIndex);
+        this.searchResult = new SearchResult(invIndex.getInvListTypeTraits(), ctx);
+        this.invIndex = invIndex;
+        this.invListCmp = MultiComparator.create(invIndex.getInvListCmpFactories());
+        this.invListCursorFactory = new InvertedListCursorFactory(invIndex);
+        this.invListCursorCache = new ObjectCache<IInvertedListCursor>(invListCursorFactory, OBJECT_CACHE_INIT_SIZE,
+                OBJECT_CACHE_EXPAND_SIZE);
+        this.queryTokenFrame = ctx.allocateFrame();
+        this.queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
+        this.queryTokenAccessor = new FrameTupleAccessor(ctx.getFrameSize(), QUERY_TOKEN_REC_DESC);
+        this.queryTokenAccessor.reset(queryTokenFrame);
+    }
+
+    public void reset() {
+        searchResult.clear();
+        invListMerger.reset();
+    }
+
+    protected void tokenizeQuery(InvertedIndexSearchPredicate searchPred) throws HyracksDataException,
+            OccurrenceThresholdPanicException {
+        ITupleReference queryTuple = searchPred.getQueryTuple();
+        int queryFieldIndex = searchPred.getQueryFieldIndex();
+        IBinaryTokenizer queryTokenizer = searchPred.getQueryTokenizer();
+
+        queryTokenAppender.reset(queryTokenFrame, true);
+        queryTokenizer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
+                queryTuple.getFieldLength(queryFieldIndex));
+
+        while (queryTokenizer.hasNext()) {
+            queryTokenizer.next();
+            queryTokenBuilder.reset();
+            try {
+                IToken token = queryTokenizer.getToken();
+                token.serializeToken(queryTokenBuilder.getFieldData());
+                queryTokenBuilder.addFieldEndOffset();
+                // WARNING: assuming one frame is big enough to hold all tokens
+                queryTokenAppender.append(queryTokenBuilder.getFieldEndOffsets(), queryTokenBuilder.getByteArray(), 0,
+                        queryTokenBuilder.getSize());
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    public IFrameTupleAccessor createResultFrameTupleAccessor() {
+        return new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), searchResult.getTypeTraits());
+    }
+
+    public ITupleReference createResultFrameTupleReference() {
+        return new FixedSizeTupleReference(searchResult.getTypeTraits());
+    }
+
+    @Override
+    public List<ByteBuffer> getResultBuffers() {
+        return searchResult.getBuffers();
+    }
+
+    @Override
+    public int getNumValidResultBuffers() {
+        return searchResult.getCurrentBufferIndex() + 1;
+    }
+
+    public int getOccurrenceThreshold() {
+        return occurrenceThreshold;
+    }
+
+    public void printNewResults(int maxResultBufIdx, List<ByteBuffer> buffer) {
+        StringBuffer strBuffer = new StringBuffer();
+        FixedSizeFrameTupleAccessor resultFrameTupleAcc = searchResult.getAccessor();
+        for (int i = 0; i <= maxResultBufIdx; i++) {
+            ByteBuffer testBuf = buffer.get(i);
+            resultFrameTupleAcc.reset(testBuf);
+            for (int j = 0; j < resultFrameTupleAcc.getTupleCount(); j++) {
+                strBuffer.append(IntegerSerializerDeserializer.getInt(resultFrameTupleAcc.getBuffer().array(),
+                        resultFrameTupleAcc.getFieldStartOffset(j, 0)) + ",");
+                strBuffer.append(IntegerSerializerDeserializer.getInt(resultFrameTupleAcc.getBuffer().array(),
+                        resultFrameTupleAcc.getFieldStartOffset(j, 1)) + " ");
+            }
+        }
+        System.out.println(strBuffer.toString());
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ArrayListFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ArrayListFactory.java
new file mode 100644
index 0000000..493063e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ArrayListFactory.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2012 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.search;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IObjectFactory;
+
+public class ArrayListFactory<T> implements IObjectFactory<ArrayList<T>>{
+    @Override
+    public ArrayList<T> create() {
+        return new ArrayList<T>();
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifier.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifier.java
new file mode 100644
index 0000000..318f1e1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifier.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+
+public class ConjunctiveSearchModifier implements IInvertedIndexSearchModifier {
+
+    @Override
+    public int getOccurrenceThreshold(int numQueryTokens) {
+        return numQueryTokens;
+    }
+
+    @Override
+    public int getNumPrefixLists(int occurrenceThreshold, int numInvLists) {
+        return 1;
+    }
+    
+    @Override
+    public String toString() {
+        return "Conjunctive Search Modifier";
+    }
+
+    @Override
+    public short getNumTokensLowerBound(short numQueryTokens) {
+        return -1;
+    }
+
+    @Override
+    public short getNumTokensUpperBound(short numQueryTokens) {
+        return -1;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifierFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifierFactory.java
new file mode 100644
index 0000000..83e1f4b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifierFactory.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class ConjunctiveSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IInvertedIndexSearchModifier createSearchModifier() {
+        return new ConjunctiveSearchModifier();
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifier.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifier.java
new file mode 100644
index 0000000..9c06f4d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifier.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+
+public class EditDistanceSearchModifier implements IInvertedIndexSearchModifier {
+
+    private int gramLength;
+    private int edThresh;
+
+    public EditDistanceSearchModifier(int gramLength, int edThresh) {
+        this.gramLength = gramLength;
+        this.edThresh = edThresh;
+    }
+
+    @Override
+    public int getOccurrenceThreshold(int numQueryTokens) {
+        return numQueryTokens - edThresh * gramLength;
+    }
+
+    @Override
+    public int getNumPrefixLists(int occurrenceThreshold, int numInvLists) {
+        return numInvLists - occurrenceThreshold + 1;
+    }
+
+    @Override
+    public short getNumTokensLowerBound(short numQueryTokens) {
+        return (short) (numQueryTokens - edThresh);
+    }
+
+    @Override
+    public short getNumTokensUpperBound(short numQueryTokens) {
+        return (short) (numQueryTokens + edThresh);
+    }
+
+    public int getGramLength() {
+        return gramLength;
+    }
+
+    public void setGramLength(int gramLength) {
+        this.gramLength = gramLength;
+    }
+
+    public int getEdThresh() {
+        return edThresh;
+    }
+
+    public void setEdThresh(int edThresh) {
+        this.edThresh = edThresh;
+    }
+
+    @Override
+    public String toString() {
+        return "Edit Distance Search Modifier, GramLen: " + gramLength + ", Threshold: " + edThresh;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifierFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifierFactory.java
new file mode 100644
index 0000000..879c34e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifierFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class EditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int gramLength;
+    private final int edThresh;
+    
+    public EditDistanceSearchModifierFactory(int gramLength, int edThresh) {
+        this.gramLength = gramLength;
+        this.edThresh = edThresh;
+    }
+    
+    @Override
+    public IInvertedIndexSearchModifier createSearchModifier() {
+        return new EditDistanceSearchModifier(gramLength, edThresh);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java
new file mode 100644
index 0000000..2065691
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+
+public class InvertedIndexSearchPredicate implements ISearchPredicate {
+    private static final long serialVersionUID = 1L;
+
+    private ITupleReference queryTuple;
+    private int queryFieldIndex;
+    private final IBinaryTokenizer queryTokenizer;
+    private final IInvertedIndexSearchModifier searchModifier;    
+    
+    public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer, IInvertedIndexSearchModifier searchModifier) {
+        this.queryTokenizer = queryTokenizer;
+        this.searchModifier = searchModifier;
+    }
+    
+    public void setQueryTuple(ITupleReference queryTuple) {
+        this.queryTuple = queryTuple;
+    }
+    
+    public ITupleReference getQueryTuple() {
+        return queryTuple;
+    }
+    
+    public void setQueryFieldIndex(int queryFieldIndex) {
+        this.queryFieldIndex = queryFieldIndex;
+    }
+    
+    public int getQueryFieldIndex() {
+        return queryFieldIndex;
+    }
+    
+    public IInvertedIndexSearchModifier getSearchModifier() {
+        return searchModifier;
+    }
+    
+    public IBinaryTokenizer getQueryTokenizer() {
+        return queryTokenizer;
+    }
+    
+    @Override
+    public MultiComparator getLowKeyComparator() {
+        // TODO: This doesn't make sense for an inverted index. Change ISearchPredicate interface.
+        return null;
+    }
+
+    @Override
+    public MultiComparator getHighKeyComparator() {
+        // TODO: This doesn't make sense for an inverted index. Change ISearchPredicate interface.
+        return null;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListCursorFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListCursorFactory.java
new file mode 100644
index 0000000..b4b3c43
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListCursorFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2012 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.search;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IObjectFactory;
+
+public class InvertedListCursorFactory implements IObjectFactory<IInvertedListCursor> {
+
+    private final IInvertedIndex invIndex;
+
+    public InvertedListCursorFactory(IInvertedIndex invIndex) {
+        this.invIndex = invIndex;
+    }
+
+    @Override
+    public IInvertedListCursor create() {
+        return invIndex.createInvertedListCursor();
+    }
+}
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
new file mode 100644
index 0000000..fbdfd64
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListMerger.java
@@ -0,0 +1,330 @@
+/*
+ * Copyright 2009-2012 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.search;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeFrameTupleAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeTupleReference;
+
+// TODO: The merge procedure is rather confusing regarding cursor positions, hasNext() calls etc.
+// Needs an overhaul some time.
+public class InvertedListMerger {
+
+    protected final MultiComparator invListCmp;
+    protected SearchResult prevSearchResult;
+    protected SearchResult newSearchResult;
+
+    public InvertedListMerger(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+        this.invListCmp = MultiComparator.createIgnoreFieldLength(invIndex.getInvListCmpFactories());
+        this.prevSearchResult = new SearchResult(invIndex.getInvListTypeTraits(), ctx);
+        this.newSearchResult = new SearchResult(prevSearchResult);
+    }
+
+    public void merge(ArrayList<IInvertedListCursor> invListCursors, int occurrenceThreshold, int numPrefixLists,
+            SearchResult searchResult) throws HyracksDataException, IndexException {
+        Collections.sort(invListCursors);
+        int numInvLists = invListCursors.size();
+        SearchResult result = null;
+        for (int i = 0; i < numInvLists; i++) {
+            SearchResult swapTemp = prevSearchResult;
+            prevSearchResult = newSearchResult;
+            newSearchResult = swapTemp;
+            newSearchResult.reset();
+            if (i + 1 != numInvLists) {
+                // Use temporary search results when not merging last list.
+                result = newSearchResult;
+            } else {
+                // When merging the last list, append results to the final search result.
+                result = searchResult;
+            }
+            IInvertedListCursor invListCursor = invListCursors.get(i);
+            invListCursor.pinPages();
+            if (i < numPrefixLists) {
+                // Merge prefix list.
+                mergePrefixList(invListCursor, prevSearchResult, result);
+            } else {
+                // Merge suffix list.
+                int numInvListElements = invListCursor.size();
+                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);
+                } else {
+                    mergeSuffixListScan(invListCursor, prevSearchResult, result, i, numInvLists,
+                            occurrenceThreshold);
+                }
+            }
+            invListCursor.unpinPages();
+        }
+    }
+
+    protected void mergeSuffixListProbe(IInvertedListCursor invListCursor, SearchResult prevSearchResult,
+            SearchResult newSearchResult, int invListIx, int numInvLists, int occurrenceThreshold)
+            throws HyracksDataException, IndexException {
+
+        int prevBufIdx = 0;
+        int maxPrevBufIdx = prevSearchResult.getCurrentBufferIndex();
+        ByteBuffer prevCurrentBuffer = prevSearchResult.getBuffers().get(0);
+
+        FixedSizeFrameTupleAccessor resultFrameTupleAcc = prevSearchResult.getAccessor();
+        FixedSizeTupleReference resultTuple = prevSearchResult.getTuple();
+
+        int resultTidx = 0;
+
+        resultFrameTupleAcc.reset(prevCurrentBuffer);
+
+        while (resultTidx < resultFrameTupleAcc.getTupleCount()) {
+
+            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
+            int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
+                    resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
+
+            if (invListCursor.containsKey(resultTuple, invListCmp)) {
+                count++;
+                newSearchResult.append(resultTuple, count);
+            } else {
+                if (count + numInvLists - invListIx > occurrenceThreshold) {
+                    newSearchResult.append(resultTuple, count);
+                }
+            }
+
+            resultTidx++;
+            if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
+                prevBufIdx++;
+                if (prevBufIdx <= maxPrevBufIdx) {
+                    prevCurrentBuffer = prevSearchResult.getBuffers().get(prevBufIdx);
+                    resultFrameTupleAcc.reset(prevCurrentBuffer);
+                    resultTidx = 0;
+                }
+            }
+        }
+    }
+
+    protected void mergeSuffixListScan(IInvertedListCursor invListCursor, SearchResult prevSearchResult,
+            SearchResult newSearchResult, int invListIx, int numInvLists, int occurrenceThreshold)
+            throws HyracksDataException, IndexException {
+
+        int prevBufIdx = 0;
+        int maxPrevBufIdx = prevSearchResult.getCurrentBufferIndex();
+        ByteBuffer prevCurrentBuffer = prevSearchResult.getBuffers().get(0);
+
+        FixedSizeFrameTupleAccessor resultFrameTupleAcc = prevSearchResult.getAccessor();
+        FixedSizeTupleReference resultTuple = prevSearchResult.getTuple();
+
+        boolean advanceCursor = true;
+        boolean advancePrevResult = false;
+        int resultTidx = 0;
+
+        resultFrameTupleAcc.reset(prevCurrentBuffer);
+
+        int invListTidx = 0;
+        int invListNumTuples = invListCursor.size();
+
+        if (invListCursor.hasNext())
+            invListCursor.next();
+
+        while (invListTidx < invListNumTuples && resultTidx < resultFrameTupleAcc.getTupleCount()) {
+
+            ITupleReference invListTuple = invListCursor.getTuple();
+
+            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
+
+            int cmp = invListCmp.compare(invListTuple, resultTuple);
+            if (cmp == 0) {
+                int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
+                        resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)) + 1;
+                newSearchResult.append(resultTuple, count);
+                advanceCursor = true;
+                advancePrevResult = true;
+            } else {
+                if (cmp < 0) {
+                    advanceCursor = true;
+                    advancePrevResult = false;
+                } else {
+                    int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
+                            resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
+                    if (count + numInvLists - invListIx > occurrenceThreshold) {
+                        newSearchResult.append(resultTuple, count);
+                    }
+                    advanceCursor = false;
+                    advancePrevResult = true;
+                }
+            }
+
+            if (advancePrevResult) {
+                resultTidx++;
+                if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
+                    prevBufIdx++;
+                    if (prevBufIdx <= maxPrevBufIdx) {
+                        prevCurrentBuffer = prevSearchResult.getBuffers().get(prevBufIdx);
+                        resultFrameTupleAcc.reset(prevCurrentBuffer);
+                        resultTidx = 0;
+                    }
+                }
+            }
+
+            if (advanceCursor) {
+                invListTidx++;
+                if (invListCursor.hasNext()) {
+                    invListCursor.next();
+                }
+            }
+        }
+
+        // append remaining elements from previous result set
+        while (resultTidx < resultFrameTupleAcc.getTupleCount()) {
+
+            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
+
+            int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
+                    resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
+            if (count + numInvLists - invListIx > occurrenceThreshold) {
+                newSearchResult.append(resultTuple, count);
+            }
+
+            resultTidx++;
+            if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
+                prevBufIdx++;
+                if (prevBufIdx <= maxPrevBufIdx) {
+                    prevCurrentBuffer = prevSearchResult.getBuffers().get(prevBufIdx);
+                    resultFrameTupleAcc.reset(prevCurrentBuffer);
+                    resultTidx = 0;
+                }
+            }
+        }
+    }
+
+    protected void mergePrefixList(IInvertedListCursor invListCursor, SearchResult prevSearchResult,
+            SearchResult newSearchResult) throws HyracksDataException, IndexException {
+
+        int prevBufIdx = 0;
+        int maxPrevBufIdx = prevSearchResult.getCurrentBufferIndex();
+        ByteBuffer prevCurrentBuffer = prevSearchResult.getBuffers().get(0);
+
+        FixedSizeFrameTupleAccessor resultFrameTupleAcc = prevSearchResult.getAccessor();
+        FixedSizeTupleReference resultTuple = prevSearchResult.getTuple();
+
+        boolean advanceCursor = true;
+        boolean advancePrevResult = false;
+        int resultTidx = 0;
+
+        resultFrameTupleAcc.reset(prevCurrentBuffer);
+
+        int invListTidx = 0;
+        int invListNumTuples = invListCursor.size();
+
+        if (invListCursor.hasNext())
+            invListCursor.next();
+
+        while (invListTidx < invListNumTuples && resultTidx < resultFrameTupleAcc.getTupleCount()) {
+
+            ITupleReference invListTuple = invListCursor.getTuple();
+            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
+
+            int cmp = invListCmp.compare(invListTuple, resultTuple);
+            if (cmp == 0) {
+                int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
+                        resultTuple.getFieldStart(resultTuple.getFieldCount() - 1)) + 1;
+                newSearchResult.append(resultTuple, count);
+                advanceCursor = true;
+                advancePrevResult = true;
+            } else {
+                if (cmp < 0) {
+                    int count = 1;
+                    newSearchResult.append(invListTuple, count);
+                    advanceCursor = true;
+                    advancePrevResult = false;
+                } else {
+                    int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
+                            resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
+                    newSearchResult.append(resultTuple, count);
+                    advanceCursor = false;
+                    advancePrevResult = true;
+                }
+            }
+
+            if (advancePrevResult) {
+                resultTidx++;
+                if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
+                    prevBufIdx++;
+                    if (prevBufIdx <= maxPrevBufIdx) {
+                        prevCurrentBuffer = prevSearchResult.getBuffers().get(prevBufIdx);
+                        resultFrameTupleAcc.reset(prevCurrentBuffer);
+                        resultTidx = 0;
+                    }
+                }
+            }
+
+            if (advanceCursor) {
+                invListTidx++;
+                if (invListCursor.hasNext()) {
+                    invListCursor.next();
+                }
+            }
+        }
+
+        // append remaining new elements from inverted list
+        while (invListTidx < invListNumTuples) {
+            ITupleReference invListTuple = invListCursor.getTuple();
+            newSearchResult.append(invListTuple, 1);
+            invListTidx++;
+            if (invListCursor.hasNext()) {
+                invListCursor.next();
+            }
+        }
+
+        // append remaining elements from previous result set
+        while (resultTidx < resultFrameTupleAcc.getTupleCount()) {
+
+            resultTuple.reset(prevCurrentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(resultTidx));
+
+            int count = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
+                    resultTuple.getFieldStart(resultTuple.getFieldCount() - 1));
+            newSearchResult.append(resultTuple, count);
+
+            resultTidx++;
+            if (resultTidx >= resultFrameTupleAcc.getTupleCount()) {
+                prevBufIdx++;
+                if (prevBufIdx <= maxPrevBufIdx) {
+                    prevCurrentBuffer = prevSearchResult.getBuffers().get(prevBufIdx);
+                    resultFrameTupleAcc.reset(prevCurrentBuffer);
+                    resultTidx = 0;
+                }
+            }
+        }
+    }
+
+    public SearchResult createSearchResult() {
+        return new SearchResult(prevSearchResult);
+    }
+
+    public void reset() {
+        prevSearchResult.clear();
+        newSearchResult.clear();
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListPartitions.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListPartitions.java
new file mode 100644
index 0000000..1b060e5
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListPartitions.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IObjectFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.ObjectCache;
+
+public class InvertedListPartitions {
+    private final int DEFAULT_NUM_PARTITIONS = 10;
+    private final int PARTITIONS_SLACK_SIZE = 10;
+    private final int OBJECT_CACHE_INIT_SIZE = 10;
+    private final int OBJECT_CACHE_EXPAND_SIZE = 10;
+    private final IObjectFactory<ArrayList<IInvertedListCursor>> arrayListFactory;
+    private final ObjectCache<ArrayList<IInvertedListCursor>> arrayListCache;
+    private ArrayList<IInvertedListCursor>[] partitions;
+    private short minValidPartitionIndex;
+    private short maxValidPartitionIndex;
+
+    public InvertedListPartitions() {
+        this.arrayListFactory = new ArrayListFactory<IInvertedListCursor>();
+        this.arrayListCache = new ObjectCache<ArrayList<IInvertedListCursor>>(arrayListFactory, OBJECT_CACHE_INIT_SIZE,
+                OBJECT_CACHE_EXPAND_SIZE);
+    }
+
+    @SuppressWarnings("unchecked")
+    public void reset(short numTokensLowerBound, short numTokensUpperBound) {
+        if (partitions == null) {
+            int initialSize;
+            if (numTokensUpperBound < 0) {
+                initialSize = DEFAULT_NUM_PARTITIONS;
+            } else {
+                initialSize = numTokensUpperBound + 1;
+            }
+            partitions = (ArrayList<IInvertedListCursor>[]) new ArrayList[initialSize];
+        } else {
+            if (numTokensUpperBound + 1 >= partitions.length) {
+                partitions = Arrays.copyOf(partitions, numTokensUpperBound + 1);
+            }
+            Arrays.fill(partitions, null);
+        }
+        arrayListCache.reset();
+        minValidPartitionIndex = Short.MAX_VALUE;
+        maxValidPartitionIndex = Short.MIN_VALUE;
+    }
+
+    public void addInvertedListCursor(IInvertedListCursor listCursor, short numTokens) {
+        if (numTokens + 1 >= partitions.length) {
+            partitions = Arrays.copyOf(partitions, numTokens + PARTITIONS_SLACK_SIZE);
+        }
+        ArrayList<IInvertedListCursor> partitionCursors = partitions[numTokens];
+        if (partitionCursors == null) {
+            partitionCursors = arrayListCache.getNext();
+            partitionCursors.clear();
+            partitions[numTokens] = partitionCursors;
+            // Update range of valid partitions.
+            if (numTokens < minValidPartitionIndex) {
+                minValidPartitionIndex = numTokens;
+            }
+            if (numTokens > maxValidPartitionIndex) {
+                maxValidPartitionIndex = numTokens;
+            }
+        }
+        partitionCursors.add(listCursor);
+    }
+
+    public ArrayList<IInvertedListCursor>[] getPartitions() {
+        return partitions;
+    }
+
+    public short getMinValidPartitionIndex() {
+        return minValidPartitionIndex;
+    }
+
+    public short getMaxValidPartitionIndex() {
+        return maxValidPartitionIndex;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifier.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifier.java
new file mode 100644
index 0000000..ede6041
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifier.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+
+public class JaccardSearchModifier implements IInvertedIndexSearchModifier {
+
+    private float jaccThresh;
+
+    public JaccardSearchModifier(float jaccThresh) {
+        this.jaccThresh = jaccThresh;
+    }
+
+    @Override
+    public int getOccurrenceThreshold(int numQueryTokens) {
+        return Math.max((int) Math.floor((float) numQueryTokens * jaccThresh), 1);
+    }
+
+    @Override
+    public int getNumPrefixLists(int occurrenceThreshold, int numInvLists) {
+        if (numInvLists == 0) {
+            return 0;
+        }
+        return numInvLists - occurrenceThreshold + 1;
+    }
+
+    @Override
+    public short getNumTokensLowerBound(short numQueryTokens) {
+        return (short) Math.floor(numQueryTokens * jaccThresh);
+    }
+
+    @Override
+    public short getNumTokensUpperBound(short numQueryTokens) {
+        return (short) Math.ceil(numQueryTokens / jaccThresh);
+    }
+
+    public float getJaccThresh() {
+        return jaccThresh;
+    }
+
+    public void setJaccThresh(float jaccThresh) {
+        this.jaccThresh = jaccThresh;
+    }
+
+    @Override
+    public String toString() {
+        return "Jaccard Search Modifier, Threshold: " + jaccThresh;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifierFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifierFactory.java
new file mode 100644
index 0000000..270862b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifierFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class JaccardSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final float jaccThresh;
+
+    public JaccardSearchModifierFactory(float jaccThresh) {
+        this.jaccThresh = jaccThresh;
+    }
+
+    @Override
+    public IInvertedIndexSearchModifier createSearchModifier() {
+        return new JaccardSearchModifier(jaccThresh);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ListEditDistanceSearchModifier.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ListEditDistanceSearchModifier.java
new file mode 100644
index 0000000..8ae22a5
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ListEditDistanceSearchModifier.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+
+public class ListEditDistanceSearchModifier implements IInvertedIndexSearchModifier {
+
+    private int edThresh;
+
+    public ListEditDistanceSearchModifier(int edThresh) {
+        this.edThresh = edThresh;
+    }
+
+    public int getEdThresh() {
+        return edThresh;
+    }
+
+    public void setEdThresh(int edThresh) {
+        this.edThresh = edThresh;
+    }
+
+    @Override
+    public int getOccurrenceThreshold(int numQueryTokens) {
+        return numQueryTokens - edThresh;
+    }
+
+    @Override
+    public int getNumPrefixLists(int occurrenceThreshold, int numInvLists) {
+        return numInvLists - occurrenceThreshold + 1;
+    }
+
+    @Override
+    public short getNumTokensLowerBound(short numQueryTokens) {
+        return (short) (numQueryTokens - edThresh);
+    }
+
+    @Override
+    public short getNumTokensUpperBound(short numQueryTokens) {
+        return (short) (numQueryTokens + edThresh);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ListEditDistanceSearchModifierFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ListEditDistanceSearchModifierFactory.java
new file mode 100644
index 0000000..d8f57b1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ListEditDistanceSearchModifierFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class ListEditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int edThresh;
+
+    public ListEditDistanceSearchModifierFactory(int edThresh) {
+        this.edThresh = edThresh;
+    }
+
+    @Override
+    public IInvertedIndexSearchModifier createSearchModifier() {
+        return new ListEditDistanceSearchModifier(edThresh);
+    }
+}
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
new file mode 100644
index 0000000..3ce1f48
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
@@ -0,0 +1,189 @@
+/*
+ * Copyright 2009-2012 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.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IPartitionedInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexSearchCursor;
+
+public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher {
+
+    protected final ArrayTupleBuilder lowerBoundTupleBuilder = new ArrayTupleBuilder(1);
+    protected final ArrayTupleReference lowerBoundTuple = new ArrayTupleReference();
+    protected final ArrayTupleBuilder upperBoundTupleBuilder = new ArrayTupleBuilder(1);
+    protected final ArrayTupleReference upperBoundTuple = new ArrayTupleReference();
+    protected final ConcatenatingTupleReference fullLowSearchKey = new ConcatenatingTupleReference(2);
+    protected final ConcatenatingTupleReference fullHighSearchKey = new ConcatenatingTupleReference(2);
+
+    // Inverted list cursors ordered by token. Used to read relevant inverted-list partitions of one token one after
+    // the other for better I/O performance (because the partitions of one inverted list are stored contiguously in a file).
+    // The above implies that we currently require holding all inverted list for a query in memory.
+    protected final ArrayList<IInvertedListCursor> cursorsOrderedByTokens = new ArrayList<IInvertedListCursor>();
+    protected final InvertedListPartitions partitions = new InvertedListPartitions();
+
+    public PartitionedTOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+        super(ctx, invIndex);
+        initHelperTuples();
+    }
+
+    private void initHelperTuples() {
+        try {
+            lowerBoundTupleBuilder.reset();
+            // Write dummy value.
+            lowerBoundTupleBuilder.getDataOutput().writeShort(Short.MIN_VALUE);
+            lowerBoundTupleBuilder.addFieldEndOffset();
+            lowerBoundTuple.reset(lowerBoundTupleBuilder.getFieldEndOffsets(), lowerBoundTupleBuilder.getByteArray());
+            // Only needed for setting the number of fields in searchKey.
+            searchKey.reset(queryTokenAccessor, 0);
+            fullLowSearchKey.reset();
+            fullLowSearchKey.addTuple(searchKey);
+            fullLowSearchKey.addTuple(lowerBoundTuple);
+
+            upperBoundTupleBuilder.reset();
+            // Write dummy value.
+            upperBoundTupleBuilder.getDataOutput().writeShort(Short.MAX_VALUE);
+            upperBoundTupleBuilder.addFieldEndOffset();
+            upperBoundTuple.reset(upperBoundTupleBuilder.getFieldEndOffsets(), upperBoundTupleBuilder.getByteArray());
+            // Only needed for setting the number of fields in searchKey.
+            searchKey.reset(queryTokenAccessor, 0);
+            fullHighSearchKey.reset();
+            fullHighSearchKey.addTuple(searchKey);
+            fullHighSearchKey.addTuple(upperBoundTuple);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    public void search(OnDiskInvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred,
+            IIndexOperationContext ictx) throws HyracksDataException, IndexException {
+        IPartitionedInvertedIndex partInvIndex = (IPartitionedInvertedIndex) invIndex;
+        searchResult.reset();
+        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);
+        cursorsOrderedByTokens.clear();
+        for (int i = 0; i < numQueryTokens; i++) {
+            searchKey.reset(queryTokenAccessor, i);
+            if (!partInvIndex.openInvertedListPartitionCursors(this, ictx, numTokensLowerBound, numTokensUpperBound,
+                    partitions, cursorsOrderedByTokens)) {
+                maxCountPossible--;
+                // No results possible.
+                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
+        // (other concurrent searchers may be in the same situation).
+        // We should detect such cases, then unpin all pages, and then keep retrying to pin until we succeed.
+        // This will require a different "tryPin()" mechanism in the BufferCache that will return false
+        // if we'd have to wait for a page to get evicted.
+        if (!cursorsOrderedByTokens.isEmpty()) {
+            for (int i = start; i <= end; i++) {
+                if (partitionCursors[i] == null) {
+                    continue;
+                }
+                // Prune partition because no element in it can satisfy the occurrence threshold.
+                if (partitionCursors[i].size() < occurrenceThreshold) {
+                    cursorsOrderedByTokens.removeAll(partitionCursors[i]);
+                }
+            }
+            // Pin all the cursors in the order of tokens.
+            int numCursors = cursorsOrderedByTokens.size();
+            for (int i = 0; i < numCursors; i++) {
+                cursorsOrderedByTokens.get(i).pinPages();
+            }
+        }
+        
+        // Process the partitions one-by-one.
+        for (int i = start; i <= end; i++) {
+            if (partitionCursors[i] == null) {
+                continue;
+            }
+            // Prune partition because no element in it can satisfy the occurrence threshold.
+            if (partitionCursors[i].size() < occurrenceThreshold) {
+                continue;
+            }
+            // Merge inverted lists of current partition.
+            int numPrefixLists = searchModifier.getNumPrefixLists(occurrenceThreshold, partitionCursors[i].size());
+            invListMerger.reset();
+            invListMerger.merge(partitionCursors[i], occurrenceThreshold, numPrefixLists, searchResult);
+        }
+        
+        resultCursor.open(null, searchPred);
+    }
+
+    public void setNumTokensBoundsInSearchKeys(short numTokensLowerBound, short numTokensUpperBound) {
+        ShortSerializerDeserializer.putShort(numTokensLowerBound, lowerBoundTuple.getFieldData(0),
+                lowerBoundTuple.getFieldStart(0));
+        ShortSerializerDeserializer.putShort(numTokensUpperBound, upperBoundTuple.getFieldData(0),
+                upperBoundTuple.getFieldStart(0));
+    }
+
+    public ITupleReference getPrefixSearchKey() {
+        return searchKey;
+    }
+
+    public ITupleReference getFullLowSearchKey() {
+        return fullLowSearchKey;
+    }
+
+    public ITupleReference getFullHighSearchKey() {
+        return fullHighSearchKey;
+    }
+
+    public IInvertedListCursor getCachedInvertedListCursor() {
+        return invListCursorCache.getNext();
+    }
+}
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
new file mode 100644
index 0000000..aa0d3f2
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
@@ -0,0 +1,182 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+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;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeFrameTupleAppender;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeTupleReference;
+
+/**
+ * Byte-buffer backed storage for intermediate and final results of inverted-index searches.
+ */
+// TODO: Rename members.
+public class SearchResult {
+    protected final ArrayList<ByteBuffer> buffers = new ArrayList<ByteBuffer>();
+    protected final IHyracksCommonContext ctx;
+    protected final FixedSizeFrameTupleAppender appender;
+    protected final FixedSizeFrameTupleAccessor accessor;
+    protected final FixedSizeTupleReference tuple;
+    protected final ITypeTraits[] typeTraits;
+    protected final int invListElementSize;
+
+    protected int currBufIdx;
+    protected int numResults;
+
+    public SearchResult(ITypeTraits[] invListFields, IHyracksCommonContext ctx) {
+        typeTraits = new ITypeTraits[invListFields.length + 1];
+        int tmp = 0;
+        for (int i = 0; i < invListFields.length; i++) {
+            typeTraits[i] = invListFields[i];
+            tmp += invListFields[i].getFixedLength();
+        }
+        invListElementSize = tmp;
+        // Integer for counting occurrences.
+        typeTraits[invListFields.length] = IntegerPointable.TYPE_TRAITS;
+        this.ctx = ctx;
+        appender = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), typeTraits);
+        accessor = new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), typeTraits);
+        tuple = new FixedSizeTupleReference(typeTraits);
+        buffers.add(ctx.allocateFrame());
+    }
+
+    /**
+     * Initialize from other search-result object to share member instances except for result buffers.
+     */
+    public SearchResult(SearchResult other) {
+        this.ctx = other.ctx;
+        this.appender = other.appender;
+        this.accessor = other.accessor;
+        this.tuple = other.tuple;
+        this.typeTraits = other.typeTraits;
+        this.invListElementSize = other.invListElementSize;
+        buffers.add(ctx.allocateFrame());
+    }
+
+    public FixedSizeFrameTupleAccessor getAccessor() {
+        return accessor;
+    }
+
+    public FixedSizeFrameTupleAppender getAppender() {
+        return appender;
+    }
+
+    public FixedSizeTupleReference getTuple() {
+        return tuple;
+    }
+
+    public ArrayList<ByteBuffer> getBuffers() {
+        return buffers;
+    }
+
+    public void reset() {
+        currBufIdx = 0;
+        numResults = 0;
+        appender.reset(buffers.get(0), true);
+    }
+
+    public void clear() {
+        currBufIdx = 0;
+        numResults = 0;
+        for (ByteBuffer buffer : buffers) {
+            appender.reset(buffer, true);
+        }
+    }
+
+    public void append(ITupleReference invListElement, int count) {
+        ByteBuffer currentBuffer = buffers.get(currBufIdx);
+        if (!appender.hasSpace()) {
+            currBufIdx++;
+            if (currBufIdx >= buffers.size()) {
+                buffers.add(ctx.allocateFrame());
+            }
+            currentBuffer = buffers.get(currBufIdx);
+            appender.reset(currentBuffer, true);
+        }
+        // Append inverted-list element.
+        if (!appender.append(invListElement.getFieldData(0), invListElement.getFieldStart(0), invListElementSize)) {
+            throw new IllegalStateException();
+        }
+        // Append count.
+        if (!appender.append(count)) {
+            throw new IllegalStateException();
+        }
+        appender.incrementTupleCount(1);
+        numResults++;
+    }
+
+    public int getCurrentBufferIndex() {
+        return currBufIdx;
+    }
+
+    public ITypeTraits[] getTypeTraits() {
+        return typeTraits;
+    }
+
+    public int getNumResults() {
+        return numResults;
+    }
+
+    // TODO: This code may help to clean up the core list-merging algorithms.
+    /*
+    public SearchResultCursor getCursor() {
+        cursor.reset();
+        return cursor;
+    }
+    
+    public class SearchResultCursor {
+        private int bufferIndex;
+        private int resultIndex;
+        private int frameResultIndex;
+        private ByteBuffer currentBuffer;
+
+        public void reset() {
+            bufferIndex = 0;
+            resultIndex = 0;
+            frameResultIndex = 0;
+            currentBuffer = buffers.get(0);
+            resultFrameTupleAcc.reset(currentBuffer);
+        }
+
+        public boolean hasNext() {
+            return resultIndex < numResults;
+        }
+
+        public void next() {
+            resultTuple.reset(currentBuffer.array(), resultFrameTupleAcc.getTupleStartOffset(frameResultIndex));            
+            if (frameResultIndex < resultFrameTupleAcc.getTupleCount()) {
+                frameResultIndex++;
+            } else {
+                bufferIndex++;
+                currentBuffer = buffers.get(bufferIndex);
+                resultFrameTupleAcc.reset(currentBuffer);
+                frameResultIndex = 0;
+            }            
+            resultIndex++;
+        }
+
+        public ITupleReference getTuple() {
+            return resultTuple;
+        }
+    }
+    */
+}
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
new file mode 100644
index 0000000..4513540
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2010 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.search;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+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.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexSearchCursor;
+
+public class TOccurrenceSearcher extends AbstractTOccurrenceSearcher {
+
+    protected final ArrayList<IInvertedListCursor> invListCursors = new ArrayList<IInvertedListCursor>();
+
+    public TOccurrenceSearcher(IHyracksCommonContext ctx, IInvertedIndex invIndex) {
+        super(ctx, invIndex);
+    }
+
+    public void search(OnDiskInvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred,
+            IIndexOperationContext ictx) throws HyracksDataException, IndexException {
+        tokenizeQuery(searchPred);
+        int numQueryTokens = queryTokenAccessor.getTupleCount();
+
+        invListCursors.clear();
+        invListCursorCache.reset();
+        for (int i = 0; i < numQueryTokens; i++) {
+            searchKey.reset(queryTokenAccessor, i);
+            IInvertedListCursor invListCursor = invListCursorCache.getNext();
+            invIndex.openInvertedListCursor(invListCursor, searchKey, ictx);
+            invListCursors.add(invListCursor);
+        }
+
+        IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier();
+        occurrenceThreshold = searchModifier.getOccurrenceThreshold(numQueryTokens);
+        if (occurrenceThreshold <= 0) {
+            throw new OccurrenceThresholdPanicException("Merge threshold is <= 0. Failing Search.");
+        }
+        int numPrefixLists = searchModifier.getNumPrefixLists(occurrenceThreshold, invListCursors.size());
+
+        searchResult.reset();
+        invListMerger.merge(invListCursors, 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/tokenizers/AbstractUTF8StringBinaryTokenizer.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
new file mode 100644
index 0000000..7c0ec4d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/AbstractUTF8StringBinaryTokenizer.java
@@ -0,0 +1,77 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+public abstract class AbstractUTF8StringBinaryTokenizer implements IBinaryTokenizer {
+
+    protected byte[] data;
+    protected int start;
+    protected int length;
+    protected int tokenLength;
+    protected int index;
+    protected int utf8Length;
+
+    protected final IntArray tokensStart;
+    protected final IntArray tokensLength;
+    protected final IToken token;
+
+    protected final boolean ignoreTokenCount;
+    protected final boolean sourceHasTypeTag;
+
+    public AbstractUTF8StringBinaryTokenizer(boolean ignoreTokenCount, boolean sourceHasTypeTag,
+            ITokenFactory tokenFactory) {
+        this.ignoreTokenCount = ignoreTokenCount;
+        this.sourceHasTypeTag = sourceHasTypeTag;
+        if (!ignoreTokenCount) {
+            tokensStart = new IntArray();
+            tokensLength = new IntArray();
+        } else {
+            tokensStart = null;
+            tokensLength = null;
+        }
+        token = tokenFactory.createToken();
+    }
+
+    @Override
+    public IToken getToken() {
+        return token;
+    }
+
+    @Override
+    public void reset(byte[] data, int start, int length) {
+        this.start = start;
+        index = this.start;
+        if (sourceHasTypeTag) {
+            index++; // skip type tag
+        }
+        utf8Length = UTF8StringPointable.getUTFLength(data, index);
+        index += 2; // skip utf8 length indicator
+        this.data = data;
+        this.length = length + start;
+
+        tokenLength = 0;
+        if (!ignoreTokenCount) {
+            tokensStart.reset();
+            tokensLength.reset();
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/AbstractUTF8Token.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/AbstractUTF8Token.java
new file mode 100644
index 0000000..c9b6e1f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/AbstractUTF8Token.java
@@ -0,0 +1,105 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
+public abstract class AbstractUTF8Token implements IToken {
+    public static final int GOLDEN_RATIO_32 = 0x09e3779b9;
+
+    protected int length;
+    protected int tokenLength;
+    protected int start;
+    protected int tokenCount;
+    protected byte[] data;
+    protected final byte tokenTypeTag;
+    protected final byte countTypeTag;
+
+    public AbstractUTF8Token() {
+        tokenTypeTag = -1;
+        countTypeTag = -1;
+    }
+
+    public AbstractUTF8Token(byte tokenTypeTag, byte countTypeTag) {
+        this.tokenTypeTag = tokenTypeTag;
+        this.countTypeTag = countTypeTag;
+    }
+
+    @Override
+    public byte[] getData() {
+        return data;
+    }
+
+    @Override
+    public int getLength() {
+        return length;
+    }
+
+    public int getLowerCaseUTF8Len(int size) {
+        int lowerCaseUTF8Len = 0;
+        int pos = start;
+        for (int i = 0; i < size; i++) {
+            char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
+            lowerCaseUTF8Len += UTF8StringPointable.getModifiedUTF8Len(c);
+            pos += UTF8StringPointable.charSize(data, pos);
+        }
+        return lowerCaseUTF8Len;
+    }
+
+    @Override
+    public int getStart() {
+        return start;
+    }
+
+    @Override
+    public int getTokenLength() {
+        return tokenLength;
+    }
+
+    public void handleCountTypeTag(DataOutput dos) throws IOException {
+        if (countTypeTag > 0) {
+            dos.write(countTypeTag);
+        }
+    }
+
+    public void handleTokenTypeTag(DataOutput dos) throws IOException {
+        if (tokenTypeTag > 0) {
+            dos.write(tokenTypeTag);
+        }
+    }
+
+    @Override
+    public void reset(byte[] data, int start, int length, int tokenLength, int tokenCount) {
+        this.data = data;
+        this.start = start;
+        this.length = length;
+        this.tokenLength = tokenLength;
+        this.tokenCount = tokenCount;
+    }
+
+    @Override
+    public void serializeTokenCount(GrowableArray out) throws IOException {
+        handleCountTypeTag(out.getDataOutput());
+        out.getDataOutput().writeInt(tokenCount);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/AbstractUTF8TokenFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/AbstractUTF8TokenFactory.java
new file mode 100644
index 0000000..1507613
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/AbstractUTF8TokenFactory.java
@@ -0,0 +1,36 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+public abstract class AbstractUTF8TokenFactory implements ITokenFactory {
+	private static final long serialVersionUID = 1L;
+	protected final byte tokenTypeTag;
+	protected final byte countTypeTag;
+
+	public AbstractUTF8TokenFactory() {
+		tokenTypeTag = -1;
+		countTypeTag = -1;
+	}
+
+	public AbstractUTF8TokenFactory(byte tokenTypeTag, byte countTypeTag) {
+		this.tokenTypeTag = tokenTypeTag;
+		this.countTypeTag = countTypeTag;
+	}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
new file mode 100644
index 0000000..4c11523
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
@@ -0,0 +1,81 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class DelimitedUTF8StringBinaryTokenizer extends AbstractUTF8StringBinaryTokenizer {
+
+    public DelimitedUTF8StringBinaryTokenizer(boolean ignoreTokenCount, boolean sourceHasTypeTag,
+            ITokenFactory tokenFactory) {
+        super(ignoreTokenCount, sourceHasTypeTag, tokenFactory);
+    }
+
+    @Override
+    public boolean hasNext() {
+        // skip delimiters
+        while (index < length && isSeparator(UTF8StringPointable.charAt(data, index))) {
+            index += UTF8StringPointable.charSize(data, index);
+        }
+        return index < length;
+    }
+
+    private boolean isSeparator(char c) {
+        return !(Character.isLetterOrDigit(c) || Character.getType(c) == Character.OTHER_LETTER || Character.getType(c) == Character.OTHER_NUMBER);
+    }
+
+    @Override
+    public void next() {
+        tokenLength = 0;
+        int currentTokenStart = index;
+        while (index < length && !isSeparator(UTF8StringPointable.charAt(data, index))) {
+            index += UTF8StringPointable.charSize(data, index);
+            tokenLength++;
+        }
+        int tokenCount = 1;
+        if (tokenLength > 0 && !ignoreTokenCount) {
+            // search if we got the same token before
+            for (int i = 0; i < tokensStart.length(); ++i) {
+                if (tokenLength == tokensLength.get(i)) {
+                    int tokenStart = tokensStart.get(i);
+                    tokenCount++; // assume we found it
+                    int offset = 0;
+                    int currLength = 0;
+                    while (currLength < tokenLength) {
+                        // case insensitive comparison
+                        if (Character.toLowerCase(UTF8StringPointable.charAt(data, currentTokenStart + offset)) != Character
+                                .toLowerCase(UTF8StringPointable.charAt(data, tokenStart + offset))) {
+                            tokenCount--;
+                            break;
+                        }
+                        offset += UTF8StringPointable.charSize(data, currentTokenStart + offset);
+                        currLength++;
+                    }
+                }
+            }
+            // add the new token to the list of seen tokens
+            tokensStart.add(currentTokenStart);
+            tokensLength.add(tokenLength);
+        }
+
+        // set token
+        token.reset(data, currentTokenStart, index, tokenLength, tokenCount);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java
new file mode 100644
index 0000000..08b962b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizerFactory.java
@@ -0,0 +1,42 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+public class DelimitedUTF8StringBinaryTokenizerFactory implements
+		IBinaryTokenizerFactory {
+
+	private static final long serialVersionUID = 1L;
+	private final boolean ignoreTokenCount;
+	private final boolean sourceHasTypeTag;
+	private final ITokenFactory tokenFactory;
+
+	public DelimitedUTF8StringBinaryTokenizerFactory(boolean ignoreTokenCount,
+			boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
+		this.ignoreTokenCount = ignoreTokenCount;
+		this.sourceHasTypeTag = sourceHasTypeTag;
+		this.tokenFactory = tokenFactory;
+	}
+
+	@Override
+	public IBinaryTokenizer createTokenizer() {
+		return new DelimitedUTF8StringBinaryTokenizer(ignoreTokenCount,
+				sourceHasTypeTag, tokenFactory);
+	}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramToken.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramToken.java
new file mode 100644
index 0000000..632bf9a
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramToken.java
@@ -0,0 +1,64 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
+public class HashedUTF8NGramToken extends UTF8NGramToken {
+    public HashedUTF8NGramToken(byte tokenTypeTag, byte countTypeTag) {
+        super(tokenTypeTag, countTypeTag);
+    }
+
+    @Override
+    public void serializeToken(GrowableArray out) throws IOException {
+        handleTokenTypeTag(out.getDataOutput());
+
+        int hash = GOLDEN_RATIO_32;
+
+        // pre chars
+        for (int i = 0; i < numPreChars; i++) {
+            hash ^= PRECHAR;
+            hash *= GOLDEN_RATIO_32;
+        }
+
+        // regular chars
+        int numRegGrams = tokenLength - numPreChars - numPostChars;
+        int pos = start;
+        for (int i = 0; i < numRegGrams; i++) {
+            hash ^= Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
+            hash *= GOLDEN_RATIO_32;
+            pos += UTF8StringPointable.charSize(data, pos);
+        }
+
+        // post chars
+        for (int i = 0; i < numPostChars; i++) {
+            hash ^= POSTCHAR;
+            hash *= GOLDEN_RATIO_32;
+        }
+
+        // token count
+        hash += tokenCount;
+
+        out.getDataOutput().writeInt(hash);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java
new file mode 100644
index 0000000..e1d8e31
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8NGramTokenFactory.java
@@ -0,0 +1,38 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+public class HashedUTF8NGramTokenFactory extends AbstractUTF8TokenFactory {
+
+	private static final long serialVersionUID = 1L;
+
+	public HashedUTF8NGramTokenFactory() {
+		super();
+	}
+
+	public HashedUTF8NGramTokenFactory(byte tokenTypeTag, byte countTypeTag) {
+		super(tokenTypeTag, countTypeTag);
+	}
+
+	@Override
+	public IToken createToken() {
+		return new HashedUTF8NGramToken(tokenTypeTag, countTypeTag);
+	}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordToken.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordToken.java
new file mode 100644
index 0000000..32954f9
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordToken.java
@@ -0,0 +1,87 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
+public class HashedUTF8WordToken extends UTF8WordToken {
+
+    private int hash = 0;
+
+    public HashedUTF8WordToken(byte tokenTypeTag, byte countTypeTag) {
+        super(tokenTypeTag, countTypeTag);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == null) {
+            return false;
+        }
+        if (!(o instanceof IToken)) {
+            return false;
+        }
+        IToken t = (IToken) o;
+        if (t.getTokenLength() != tokenLength) {
+            return false;
+        }
+        int offset = 0;
+        for (int i = 0; i < tokenLength; i++) {
+            if (UTF8StringPointable.charAt(t.getData(), t.getStart() + offset) != UTF8StringPointable.charAt(data,
+                    start + offset)) {
+                return false;
+            }
+            offset += UTF8StringPointable.charSize(data, start + offset);
+        }
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        return hash;
+    }
+
+    @Override
+    public void reset(byte[] data, int start, int length, int tokenLength, int tokenCount) {
+        super.reset(data, start, length, tokenLength, tokenCount);
+
+        // pre-compute hash value using JAQL-like string hashing
+        int pos = start;
+        hash = GOLDEN_RATIO_32;
+        for (int i = 0; i < tokenLength; i++) {
+            hash ^= Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
+            hash *= GOLDEN_RATIO_32;
+            pos += UTF8StringPointable.charSize(data, pos);
+        }
+        hash += tokenCount;
+    }
+
+    @Override
+    public void serializeToken(GrowableArray out) throws IOException {
+        if (tokenTypeTag > 0) {
+            out.getDataOutput().write(tokenTypeTag);
+        }
+
+        // serialize hash value
+        out.getDataOutput().writeInt(hash);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java
new file mode 100644
index 0000000..a4788c4
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/HashedUTF8WordTokenFactory.java
@@ -0,0 +1,38 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+public class HashedUTF8WordTokenFactory extends AbstractUTF8TokenFactory {
+
+	private static final long serialVersionUID = 1L;
+
+	public HashedUTF8WordTokenFactory() {
+		super();
+	}
+
+	public HashedUTF8WordTokenFactory(byte tokenTypeTag, byte countTypeTag) {
+		super(tokenTypeTag, countTypeTag);
+	}
+
+	@Override
+	public IToken createToken() {
+		return new HashedUTF8WordToken(tokenTypeTag, countTypeTag);
+	}
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
new file mode 100644
index 0000000..f88e744
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
@@ -0,0 +1,30 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+public interface IBinaryTokenizer {
+	public IToken getToken();
+
+	public boolean hasNext();
+
+	public void next();
+
+	public void reset(byte[] data, int start, int length);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizerFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizerFactory.java
new file mode 100644
index 0000000..5890124
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizerFactory.java
@@ -0,0 +1,26 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import java.io.Serializable;
+
+public interface IBinaryTokenizerFactory extends Serializable {
+	public IBinaryTokenizer createTokenizer();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/INGramToken.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/INGramToken.java
new file mode 100644
index 0000000..40351c4
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/INGramToken.java
@@ -0,0 +1,28 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+public interface INGramToken {
+	public int getNumPostChars();
+
+	public int getNumPreChars();
+
+	public void setNumPrePostChars(int numPreChars, int numPostChars);
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IToken.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IToken.java
new file mode 100644
index 0000000..7b1a130
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IToken.java
@@ -0,0 +1,41 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
+public interface IToken {
+	public byte[] getData();
+
+	public int getLength();
+
+	public int getStart();
+
+	public int getTokenLength();
+
+	public void reset(byte[] data, int start, int length, int tokenLength,
+			int tokenCount);
+
+	public void serializeToken(GrowableArray out) throws IOException;
+
+	public void serializeTokenCount(GrowableArray out) throws IOException;
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/ITokenFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/ITokenFactory.java
new file mode 100644
index 0000000..5765263
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/ITokenFactory.java
@@ -0,0 +1,26 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import java.io.Serializable;
+
+public interface ITokenFactory extends Serializable {
+    public IToken createToken();
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IntArray.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IntArray.java
new file mode 100644
index 0000000..6bae90b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/IntArray.java
@@ -0,0 +1,80 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Rares Vernica <rares (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import java.util.Arrays;
+
+public class IntArray {
+    private static final int SIZE = 128;
+
+    private int[] data;
+    private int length;
+
+    public IntArray() {
+        data = new int[SIZE];
+        length = 0;
+    }
+
+    public void add(int d) {
+        if (length == data.length) {
+            data = Arrays.copyOf(data, data.length << 1);
+        }
+        data[length++] = d;
+    }
+
+    public int[] get() {
+        return data;
+    }
+
+    public int get(int i) {
+        return data[i];
+    }
+
+    public int length() {
+        return length;
+    }
+
+    public void reset() {
+        length = 0;
+    }
+
+    public void sort() {
+        sort(0, length);
+    }
+
+    public void sort(int start, int end) {
+        Arrays.sort(data, start, end);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder out = new StringBuilder();
+        out.append('[');
+        for (int i = 0; i < length; ++i) {
+            out.append(data[i]);
+            if (i < length - 1) {
+                out.append(',');
+                out.append(' ');
+            }
+        }
+        out.append(']');
+        return out.toString();
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
new file mode 100644
index 0000000..0af0335
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
@@ -0,0 +1,118 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class NGramUTF8StringBinaryTokenizer extends AbstractUTF8StringBinaryTokenizer {
+
+    private int gramLength;
+    private boolean usePrePost;
+
+    private int gramNum;
+    private int totalGrams;
+
+    private final INGramToken concreteToken;
+
+    public NGramUTF8StringBinaryTokenizer(int gramLength, boolean usePrePost, boolean ignoreTokenCount,
+            boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
+        super(ignoreTokenCount, sourceHasTypeTag, tokenFactory);
+        this.gramLength = gramLength;
+        this.usePrePost = usePrePost;
+        concreteToken = (INGramToken) token;
+    }
+
+    @Override
+    public boolean hasNext() {
+        if (gramNum < totalGrams) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public void next() {
+        int currentTokenStart = index;
+        int tokenCount = 1;
+        int numPreChars = 0;
+        int numPostChars = 0;
+        if (usePrePost) {
+            numPreChars = Math.max(gramLength - gramNum - 1, 0);
+            numPostChars = (gramNum > totalGrams - gramLength) ? gramLength - totalGrams + gramNum : 0;
+        }
+        gramNum++;
+
+        concreteToken.setNumPrePostChars(numPreChars, numPostChars);
+        if (numPreChars == 0) {
+            index += UTF8StringPointable.charSize(data, index);
+        }
+
+        // compute token count
+        // ignore pre and post grams for duplicate detection
+        if (!ignoreTokenCount && numPreChars == 0 && numPostChars == 0) {
+            int tmpIndex = start;
+            while (tmpIndex < currentTokenStart) {
+                tokenCount++; // assume found
+                int offset = 0;
+                for (int j = 0; j < gramLength; j++) {
+                    if (Character.toLowerCase(UTF8StringPointable.charAt(data, currentTokenStart + offset)) != Character
+                            .toLowerCase(UTF8StringPointable.charAt(data, tmpIndex + offset))) {
+                        tokenCount--;
+                        break;
+                    }
+                    offset += UTF8StringPointable.charSize(data, tmpIndex + offset);
+                }
+                tmpIndex += UTF8StringPointable.charSize(data, tmpIndex);
+            }
+        }
+
+        // set token
+        token.reset(data, currentTokenStart, length, gramLength, tokenCount);
+    }
+
+    @Override
+    public void reset(byte[] data, int start, int length) {
+        super.reset(data, start, length);
+        gramNum = 0;
+
+        int numChars = 0;
+        int pos = index;
+        int end = pos + utf8Length;
+        while (pos < end) {
+            numChars++;
+            pos += UTF8StringPointable.charSize(data, pos);
+        }
+
+        if (usePrePost) {
+            totalGrams = numChars + gramLength - 1;
+        } else {
+            totalGrams = numChars - gramLength + 1;
+        }
+    }
+
+    public void setGramlength(int gramLength) {
+        this.gramLength = gramLength;
+    }
+
+    public void setPrePost(boolean usePrePost) {
+        this.usePrePost = usePrePost;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizerFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizerFactory.java
new file mode 100644
index 0000000..da3d411
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizerFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2010 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.tokenizers;
+
+public class NGramUTF8StringBinaryTokenizerFactory implements IBinaryTokenizerFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final int gramLength;
+    private final boolean usePrePost;
+    private final boolean ignoreTokenCount;
+    private final boolean sourceHasTypeTag;
+    private final ITokenFactory tokenFactory;
+
+    public NGramUTF8StringBinaryTokenizerFactory(int gramLength, boolean usePrePost, boolean ignoreTokenCount,
+            boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
+        this.gramLength = gramLength;
+        this.usePrePost = usePrePost;
+        this.ignoreTokenCount = ignoreTokenCount;
+        this.sourceHasTypeTag = sourceHasTypeTag;
+        this.tokenFactory = tokenFactory;
+    }
+
+    @Override
+    public IBinaryTokenizer createTokenizer() {
+        return new NGramUTF8StringBinaryTokenizer(gramLength, usePrePost, ignoreTokenCount, sourceHasTypeTag,
+                tokenFactory);
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramToken.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramToken.java
new file mode 100644
index 0000000..8713499
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramToken.java
@@ -0,0 +1,91 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+
+public class UTF8NGramToken extends AbstractUTF8Token implements INGramToken {
+
+    public final static char PRECHAR = '#';
+
+    public final static char POSTCHAR = '$';
+
+    protected int numPreChars;
+    protected int numPostChars;
+
+    public UTF8NGramToken(byte tokenTypeTag, byte countTypeTag) {
+        super(tokenTypeTag, countTypeTag);
+    }
+
+    @Override
+    public int getNumPostChars() {
+        return numPreChars;
+    }
+
+    @Override
+    public int getNumPreChars() {
+        return numPostChars;
+    }
+
+    @Override
+    public void serializeToken(GrowableArray out) throws IOException {
+        handleTokenTypeTag(out.getDataOutput());
+        int tokenUTF8LenOff = out.getLength();
+
+        // regular chars
+        int numRegChars = tokenLength - numPreChars - numPostChars;
+
+        // assuming pre and post char need 1-byte each in utf8
+        int tokenUTF8Len = numPreChars + numPostChars;
+
+        // Write dummy UTF length which will be correctly set later.
+        out.getDataOutput().writeShort(0);
+
+        // pre chars
+        for (int i = 0; i < numPreChars; i++) {
+            StringUtils.writeCharAsModifiedUTF8(PRECHAR, out.getDataOutput());
+        }
+
+        int pos = start;
+        for (int i = 0; i < numRegChars; i++) {
+            char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
+            tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
+            pos += UTF8StringPointable.charSize(data, pos);
+        }
+
+        // post chars
+        for (int i = 0; i < numPostChars; i++) {
+            StringUtils.writeCharAsModifiedUTF8(POSTCHAR, out.getDataOutput());
+        }
+
+        // Set UTF length of token.
+        out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
+        out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
+    }
+
+    public void setNumPrePostChars(int numPreChars, int numPostChars) {
+        this.numPreChars = numPreChars;
+        this.numPostChars = numPostChars;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramTokenFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramTokenFactory.java
new file mode 100644
index 0000000..d26a409
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8NGramTokenFactory.java
@@ -0,0 +1,39 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+public class UTF8NGramTokenFactory extends AbstractUTF8TokenFactory {
+
+	private static final long serialVersionUID = 1L;
+
+	public UTF8NGramTokenFactory() {
+		super();
+	}
+
+	public UTF8NGramTokenFactory(byte tokenTypeTag, byte countTypeTag) {
+		super(tokenTypeTag, countTypeTag);
+	}
+
+	@Override
+	public IToken createToken() {
+		return new UTF8NGramToken(tokenTypeTag, countTypeTag);
+	}
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordToken.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordToken.java
new file mode 100644
index 0000000..dbfc76f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordToken.java
@@ -0,0 +1,51 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
+
+public class UTF8WordToken extends AbstractUTF8Token {
+
+    public UTF8WordToken(byte tokenTypeTag, byte countTypeTag) {
+        super(tokenTypeTag, countTypeTag);
+    }
+
+    @Override
+    public void serializeToken(GrowableArray out) throws IOException {
+        handleTokenTypeTag(out.getDataOutput());
+        int tokenUTF8LenOff = out.getLength();
+        int tokenUTF8Len = 0;
+        // Write dummy UTF length which will be correctly set later.
+        out.getDataOutput().writeShort(0);
+        int pos = start;
+        for (int i = 0; i < tokenLength; i++) {
+            char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
+            tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
+            pos += UTF8StringPointable.charSize(data, pos);
+        }
+        // Set UTF length of token.
+        out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
+        out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordTokenFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordTokenFactory.java
new file mode 100644
index 0000000..023e957
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/UTF8WordTokenFactory.java
@@ -0,0 +1,39 @@
+/**
+ * Copyright 2010-2011 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 at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on
+ * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under
+ * the License.
+ * 
+ * Author: Alexander Behm <abehm (at) ics.uci.edu>
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers;
+
+public class UTF8WordTokenFactory extends AbstractUTF8TokenFactory {
+
+	private static final long serialVersionUID = 1L;
+
+	public UTF8WordTokenFactory() {
+		super();
+	}
+
+	public UTF8WordTokenFactory(byte tokenTypeTag, byte countTypeTag) {
+		super(tokenTypeTag, countTypeTag);
+	}
+
+	@Override
+	public IToken createToken() {
+		return new UTF8WordToken(tokenTypeTag, countTypeTag);
+	}
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java
new file mode 100644
index 0000000..b9f9362
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2009-2012 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.util;
+
+import java.io.IOException;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+
+// TODO: We can possibly avoid copying the data into a new tuple here.
+public class InvertedIndexTokenizingTupleIterator {
+    // Field that is expected to be tokenized.
+    protected final int DOC_FIELD_INDEX = 0;
+
+    protected final int invListFieldCount;
+    protected final ArrayTupleBuilder tupleBuilder;
+    protected final ArrayTupleReference tupleReference;
+    protected final IBinaryTokenizer tokenizer;
+    protected ITupleReference inputTuple;
+
+    public InvertedIndexTokenizingTupleIterator(int tokensFieldCount, int invListFieldCount, IBinaryTokenizer tokenizer) {
+        this.invListFieldCount = invListFieldCount;
+        this.tupleBuilder = new ArrayTupleBuilder(tokensFieldCount + invListFieldCount);
+        this.tupleReference = new ArrayTupleReference();
+        this.tokenizer = tokenizer;
+    }
+
+    public void reset(ITupleReference inputTuple) {
+        this.inputTuple = inputTuple;
+        tokenizer.reset(inputTuple.getFieldData(DOC_FIELD_INDEX), inputTuple.getFieldStart(DOC_FIELD_INDEX),
+                inputTuple.getFieldLength(DOC_FIELD_INDEX));
+    }
+
+    public boolean hasNext() {
+        return tokenizer.hasNext();
+    }
+
+    public void next() throws HyracksDataException {
+        tokenizer.next();
+        IToken token = tokenizer.getToken();
+        tupleBuilder.reset();
+        // Add token field.
+        try {
+            token.serializeToken(tupleBuilder.getFieldData());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+        tupleBuilder.addFieldEndOffset();
+        // Add inverted-list element fields.
+        for (int i = 0; i < invListFieldCount; i++) {
+            tupleBuilder.addField(inputTuple.getFieldData(i + 1), inputTuple.getFieldStart(i + 1),
+                    inputTuple.getFieldLength(i + 1));
+        }
+        // Reset tuple reference for insert operation.
+        tupleReference.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+    }
+
+    public ITupleReference getTuple() {
+        return tupleReference;
+    }
+}
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
new file mode 100644
index 0000000..79c8ccf
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -0,0 +1,218 @@
+/*
+ * Copyright 2009-2010 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.util;
+
+import java.io.File;
+
+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;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.PartitionedLSMInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.PartitionedInMemoryInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeElementInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeElementInvertedListBuilderFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.PartitionedOnDiskInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.PartitionedOnDiskInvertedIndexFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class InvertedIndexUtils {
+
+    public static InMemoryInvertedIndex createInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
+            IFreePageManager memFreePageManager, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory)
+            throws BTreeException {
+        return new InMemoryInvertedIndex(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories,
+                tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
+    }
+
+    public static InMemoryInvertedIndex createPartitionedInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
+            IFreePageManager memFreePageManager, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory)
+            throws BTreeException {
+        return new PartitionedInMemoryInvertedIndex(memBufferCache, memFreePageManager, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
+    }
+
+    public static OnDiskInvertedIndex createOnDiskInvertedIndex(IBufferCache bufferCache,
+            IFileMapProvider fileMapProvider, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile) throws IndexException {
+        IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
+        FileReference btreeFile = getBTreeFile(invListsFile);
+        return new OnDiskInvertedIndex(bufferCache, fileMapProvider, builder, invListTypeTraits, invListCmpFactories,
+                tokenTypeTraits, tokenCmpFactories, btreeFile, invListsFile);
+    }
+
+    public static PartitionedOnDiskInvertedIndex createPartitionedOnDiskInvertedIndex(IBufferCache bufferCache,
+            IFileMapProvider fileMapProvider, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, FileReference invListsFile) throws IndexException {
+        IInvertedListBuilder builder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
+        FileReference btreeFile = getBTreeFile(invListsFile);
+        return new PartitionedOnDiskInvertedIndex(bufferCache, fileMapProvider, builder, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, btreeFile, invListsFile);
+    }
+
+    public static FileReference getBTreeFile(FileReference invListsFile) {
+        return new FileReference(new File(invListsFile.getFile().getPath() + "_btree"));
+    }
+
+    public static BTreeFactory createDeletedKeysBTreeFactory(IFileMapProvider diskFileMapProvider,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            IBufferCache diskBufferCache) throws BTreeException {
+        TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(invListTypeTraits);
+        ITreeIndexFrameFactory leafFrameFactory = BTreeUtils.getLeafFrameFactory(tupleWriterFactory,
+                BTreeLeafFrameType.REGULAR_NSM);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        LinkedListFreePageManagerFactory freePageManagerFactory = new LinkedListFreePageManagerFactory(diskBufferCache,
+                metaFrameFactory);
+        BTreeFactory deletedKeysBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, interiorFrameFactory, leafFrameFactory, invListCmpFactories,
+                invListCmpFactories.length);
+        return deletedKeysBTreeFactory;
+    }
+
+    public static LSMInvertedIndex createLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
+            String onDiskDir, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+            throws IndexException {
+        return createLSMInvertedIndex(memBufferCache, memFreePageManager, diskFileMapProvider, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, diskBufferCache, ioManager,
+                onDiskDir, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, 0);
+    }
+
+    public static LSMInvertedIndex createLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
+            String onDiskDir, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            int startIODeviceIndex) throws IndexException {
+
+        BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
+                invListCmpFactories, diskBufferCache);
+
+        int[] bloomFilterKeyFields = new int[invListCmpFactories.length];
+        for (int i = 0; i < invListCmpFactories.length; i++) {
+            bloomFilterKeyFields[i] = i;
+        }
+        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
+                bloomFilterKeyFields);
+
+        FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
+        LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
+                onDiskDirFileRef, deletedKeysBTreeFactory, startIODeviceIndex);
+
+        IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
+                invListTypeTraits);
+        OnDiskInvertedIndexFactory invIndexFactory = new OnDiskInvertedIndexFactory(diskBufferCache,
+                diskFileMapProvider, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                tokenCmpFactories, fileManager);
+
+        LSMInvertedIndex invIndex = new LSMInvertedIndex(memBufferCache, memFreePageManager, invIndexFactory,
+                deletedKeysBTreeFactory, bloomFilterFactory, fileManager, diskFileMapProvider, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        return invIndex;
+    }
+
+    public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
+            String onDiskDir, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+            throws IndexException {
+        return createPartitionedLSMInvertedIndex(memBufferCache, memFreePageManager, diskFileMapProvider,
+                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                diskBufferCache, ioManager, onDiskDir, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider, 0);
+    }
+
+    public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
+            String onDiskDir, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            int startIODeviceIndex) throws IndexException {
+
+        BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
+                invListCmpFactories, diskBufferCache);
+
+        int[] bloomFilterKeyFields = new int[invListCmpFactories.length];
+        for (int i = 0; i < invListCmpFactories.length; i++) {
+            bloomFilterKeyFields[i] = i;
+        }
+        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
+                bloomFilterKeyFields);
+
+        FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
+        LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
+                onDiskDirFileRef, deletedKeysBTreeFactory, startIODeviceIndex);
+
+        IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
+                invListTypeTraits);
+        PartitionedOnDiskInvertedIndexFactory invIndexFactory = new PartitionedOnDiskInvertedIndexFactory(
+                diskBufferCache, diskFileMapProvider, invListBuilderFactory, invListTypeTraits, invListCmpFactories,
+                tokenTypeTraits, tokenCmpFactories, fileManager);
+
+        PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(memBufferCache, memFreePageManager,
+                invIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory, fileManager, diskFileMapProvider,
+                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        return invIndex;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/ObjectCache.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/ObjectCache.java
new file mode 100644
index 0000000..b073f20
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/ObjectCache.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2012 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.util;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IObjectFactory;
+
+public class ObjectCache<T> {
+    protected final int expandSize;
+    protected final IObjectFactory<T> objFactory;
+    protected final ArrayList<T> cache;
+    protected int lastReturned = 0;
+
+    public ObjectCache(IObjectFactory<T> objFactory, int initialSize, int expandSize) {
+        this.objFactory = objFactory;
+        this.cache = new ArrayList<T>(initialSize);
+        this.expandSize = expandSize;
+        expand(initialSize);
+    }
+
+    private void expand(int expandSize) {
+        for (int i = 0; i < expandSize; i++) {
+            cache.add(objFactory.create());
+        }
+    }
+
+    public void reset() {
+        lastReturned = 0;
+    }
+
+    public T getNext() {
+        if (lastReturned >= cache.size()) {
+            expand(expandSize);
+        }
+        return cache.get(lastReturned++);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
new file mode 100644
index 0000000..8a18a91
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2012 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.util;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+
+// TODO: We can possibly avoid copying the data into a new tuple here.
+public class PartitionedInvertedIndexTokenizingTupleIterator extends InvertedIndexTokenizingTupleIterator {
+
+    protected short numTokens = 0;
+
+    public PartitionedInvertedIndexTokenizingTupleIterator(int tokensFieldCount, int invListFieldCount,
+            IBinaryTokenizer tokenizer) {
+        super(tokensFieldCount, invListFieldCount, tokenizer);
+    }
+
+    public void reset(ITupleReference inputTuple) {
+        super.reset(inputTuple);
+        // Run through the tokenizer once to get the total number of tokens.
+        numTokens = 0;
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+            numTokens++;
+        }
+        super.reset(inputTuple);
+    }
+
+    public void next() throws HyracksDataException {
+        tokenizer.next();
+        IToken token = tokenizer.getToken();
+        tupleBuilder.reset();
+        try {
+            // Add token field.
+            token.serializeToken(tupleBuilder.getFieldData());
+            tupleBuilder.addFieldEndOffset();
+            // Add field with number of tokens.
+            tupleBuilder.getDataOutput().writeShort(numTokens);
+            tupleBuilder.addFieldEndOffset();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+        // Add inverted-list element fields.
+        for (int i = 0; i < invListFieldCount; i++) {
+            tupleBuilder.addField(inputTuple.getFieldData(i + 1), inputTuple.getFieldStart(i + 1),
+                    inputTuple.getFieldLength(i + 1));
+        }
+        // Reset tuple reference for insert operation.
+        tupleReference.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+    }
+
+    public short getNumTokens() {
+        return numTokens;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/pom.xml b/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
new file mode 100644
index 0000000..4b2ce55
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
@@ -0,0 +1,47 @@
+<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>
+  <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-lsm-common</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-btree</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-rtree</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>  		
+  </dependencies>
+</project>
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
new file mode 100644
index 0000000..c363c99
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+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.ITreeIndexMetaDataFrameFactory;
+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.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public abstract class AbstractLSMRTreeDataflowHelper extends AbstractLSMIndexDataflowHelper {
+
+    protected final IBinaryComparatorFactory[] btreeComparatorFactories;
+    protected final IPrimitiveValueProviderFactory[] valueProviderFactories;
+    protected final RTreePolicyType rtreePolicyType;
+    protected final ILinearizeComparatorFactory linearizeCmpFactory;
+
+    public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            IBinaryComparatorFactory[] btreeComparatorFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory) {
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, btreeComparatorFactories,
+                valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider, linearizeCmpFactory);
+    }
+
+    public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            int memPageSize, int memNumPages, IBinaryComparatorFactory[] btreeComparatorFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory) {
+        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
+        this.btreeComparatorFactories = btreeComparatorFactories;
+        this.valueProviderFactories = valueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
+        this.linearizeCmpFactory = linearizeCmpFactory;
+    }
+
+    @Override
+    public ITreeIndex createIndexInstance() throws HyracksDataException {
+        AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
+        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+        IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
+                memNumPages);
+        IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
+                metaDataFrameFactory);
+        return createLSMTree(memBufferCache, memFreePageManager, ctx.getIOManager(), file, opDesc.getStorageManager()
+                .getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx),
+                treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(),
+                btreeComparatorFactories, valueProviderFactories, rtreePolicyType, linearizeCmpFactory, partition);
+
+    }
+
+    protected abstract ITreeIndex createLSMTree(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) 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
new file mode 100644
index 0000000..1df914e
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMRTreeDataflowHelper extends AbstractLSMRTreeDataflowHelper {
+
+    public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            IBinaryComparatorFactory[] btreeComparatorFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory) {
+        super(opDesc, ctx, partition, btreeComparatorFactories, valueProviderFactories, rtreePolicyType, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+    }
+
+    public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+            int memPageSize, int memNumPages, IBinaryComparatorFactory[] btreeComparatorFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory) {
+        super(opDesc, ctx, partition, memPageSize, memNumPages, btreeComparatorFactories, valueProviderFactories,
+                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+    }
+
+    @Override
+    protected ITreeIndex createLSMTree(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException {
+        try {
+            return LSMRTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache,
+                    diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
+                    rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+                    linearizeCmpFactory, startIODeviceIndex);
+        } 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/LSMRTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
new file mode 100644
index 0000000..a730895
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.dataflow.AbstractLSMIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class LSMRTreeDataflowHelperFactory extends AbstractLSMIndexDataflowHelperFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final IBinaryComparatorFactory[] btreeComparatorFactories;
+    private final IPrimitiveValueProviderFactory[] valueProviderFactories;
+    private final RTreePolicyType rtreePolicyType;
+    private final ILinearizeComparatorFactory linearizeCmpFactory;
+
+    public LSMRTreeDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
+            ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory, int memPageSize, int memNumPages) {
+        super(mergePolicyProvider, opTrackerFactory, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
+                memNumPages);
+        this.btreeComparatorFactories = btreeComparatorFactories;
+        this.valueProviderFactories = valueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
+        this.linearizeCmpFactory = linearizeCmpFactory;
+    }
+
+    @Override
+    public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition) {
+        return new LSMRTreeDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages, btreeComparatorFactories, valueProviderFactories,
+                rtreePolicyType, mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory,
+                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider, linearizeCmpFactory);
+    }
+}
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
new file mode 100644
index 0000000..6f5ecb1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMRTreeWithAntiMatterTuplesDataflowHelper extends AbstractLSMRTreeDataflowHelper {
+    public LSMRTreeWithAntiMatterTuplesDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, IBinaryComparatorFactory[] btreeComparatorFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory) {
+        super(opDesc, ctx, partition, btreeComparatorFactories, valueProviderFactories, rtreePolicyType, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+    }
+
+    public LSMRTreeWithAntiMatterTuplesDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition, int memPageSize, int memNumPages, IBinaryComparatorFactory[] btreeComparatorFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory) {
+        super(opDesc, ctx, partition, memPageSize, memNumPages, btreeComparatorFactories, valueProviderFactories,
+                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+    }
+
+    @Override
+    protected ITreeIndex createLSMTree(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws HyracksDataException {
+        try {
+            return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(memBufferCache, memFreePageManager, ioManager, file,
+                    diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                    valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
+                    ioOpCallbackProvider, linearizeCmpFactory, startIODeviceIndex);
+        } 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/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
new file mode 100644
index 0000000..b27e84f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class LSMRTreeWithAntiMatterTuplesDataflowHelperFactory implements IIndexDataflowHelperFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final IBinaryComparatorFactory[] btreeComparatorFactories;
+    private final IPrimitiveValueProviderFactory[] valueProviderFactories;
+    private final RTreePolicyType rtreePolicyType;
+    private final ILSMMergePolicyProvider mergePolicyProvider;
+    private final ILSMOperationTrackerFactory opTrackerProvider;
+    private final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
+    private final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    private final ILinearizeComparatorFactory linearizeCmpFactory;
+
+    public LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
+            ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerFactory opTrackerProvider,
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory) {
+        this.btreeComparatorFactories = btreeComparatorFactories;
+        this.valueProviderFactories = valueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
+        this.mergePolicyProvider = mergePolicyProvider;
+        this.ioSchedulerProvider = ioSchedulerProvider;
+        this.opTrackerProvider = opTrackerProvider;
+        this.ioOpCallbackProvider = ioOpCallbackProvider;
+        this.linearizeCmpFactory = linearizeCmpFactory;
+    }
+
+    @Override
+    public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+            int partition) {
+        return new LSMRTreeWithAntiMatterTuplesDataflowHelper(opDesc, ctx, partition, btreeComparatorFactories,
+                valueProviderFactories, rtreePolicyType, mergePolicyProvider.getMergePolicy(ctx), opTrackerProvider,
+                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider, linearizeCmpFactory);
+    }
+}
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
new file mode 100644
index 0000000..23137ab
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -0,0 +1,357 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.io.File;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeNonExistentKeyException;
+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.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public abstract class AbstractLSMRTree extends AbstractLSMIndex implements ITreeIndex {
+
+    protected final ILinearizeComparatorFactory linearizer;
+    protected final int[] comparatorFields;
+    protected final IBinaryComparatorFactory[] linearizerArray;
+
+    // In-memory components.
+    protected final LSMRTreeMutableComponent mutableComponent;
+    protected final IInMemoryBufferCache memBufferCache;
+
+    protected TreeTupleSorter rTreeTupleSorter;
+
+    // On-disk components.
+    // For creating RTree's used in flush and merge.
+    protected final ILSMComponentFactory componentFactory;
+
+    private IBinaryComparatorFactory[] btreeCmpFactories;
+    private IBinaryComparatorFactory[] rtreeCmpFactories;
+
+    // Common for in-memory and on-disk components.
+    protected final ITreeIndexFrameFactory rtreeInteriorFrameFactory;
+    protected final ITreeIndexFrameFactory btreeInteriorFrameFactory;
+    protected final ITreeIndexFrameFactory rtreeLeafFrameFactory;
+    protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
+
+    public AbstractLSMRTree(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
+            ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory rtreeLeafFrameFactory,
+            ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
+            ILSMIndexFileManager fileManager, TreeIndexFactory<RTree> diskRTreeFactory,
+            ILSMComponentFactory componentFactory, IFileMapProvider diskFileMapProvider, int fieldCount,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(memFreePageManager, diskRTreeFactory.getBufferCache(), fileManager, diskFileMapProvider, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        RTree memRTree = new RTree(memBufferCache, ((InMemoryBufferCache) memBufferCache).getFileMapProvider(),
+                memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, rtreeCmpFactories, fieldCount,
+                new FileReference(new File("memrtree")));
+        BTree memBTree = new BTree(memBufferCache, ((InMemoryBufferCache) memBufferCache).getFileMapProvider(),
+                memFreePageManager, btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeCmpFactories, fieldCount,
+                new FileReference(new File("membtree")));
+        mutableComponent = new LSMRTreeMutableComponent(memRTree, memBTree, memFreePageManager);
+        this.memBufferCache = memBufferCache;
+        this.rtreeInteriorFrameFactory = rtreeInteriorFrameFactory;
+        this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
+        this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
+        this.btreeLeafFrameFactory = btreeLeafFrameFactory;
+        this.componentFactory = componentFactory;
+        this.btreeCmpFactories = btreeCmpFactories;
+        this.rtreeCmpFactories = rtreeCmpFactories;
+        this.linearizer = linearizer;
+        this.comparatorFields = comparatorFields;
+        this.linearizerArray = linearizerArray;
+        rTreeTupleSorter = null;
+    }
+
+    @Override
+    public synchronized void create() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to create the index since it is activated.");
+        }
+
+        fileManager.deleteDirs();
+        fileManager.createDirs();
+        componentsRef.get().clear();
+    }
+
+    @Override
+    public synchronized void activate() throws HyracksDataException {
+        if (isActivated) {
+            return;
+        }
+
+        ((InMemoryBufferCache) mutableComponent.getRTree().getBufferCache()).open();
+        mutableComponent.getRTree().create();
+        mutableComponent.getBTree().create();
+        mutableComponent.getRTree().activate();
+        mutableComponent.getBTree().activate();
+    }
+
+    @Override
+    public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
+        if (!isActivated) {
+            return;
+        }
+
+        if (flushOnExit) {
+            BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
+                    ioOpCallbackProvider.getIOOperationCallback(this));
+            ILSMIndexAccessor accessor = (ILSMIndexAccessor) createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            accessor.scheduleFlush(cb);
+            try {
+                cb.waitForIO();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        mutableComponent.getRTree().deactivate();
+        mutableComponent.getBTree().deactivate();
+        mutableComponent.getRTree().destroy();
+        mutableComponent.getBTree().destroy();
+        ((InMemoryBufferCache) mutableComponent.getRTree().getBufferCache()).close();
+    }
+
+    @Override
+    public synchronized void destroy() throws HyracksDataException {
+        if (isActivated) {
+            throw new HyracksDataException("Failed to destroy the index since it is activated.");
+        }
+
+        mutableComponent.getRTree().deactivate();
+        mutableComponent.getBTree().deactivate();
+    }
+
+    @Override
+    public synchronized void clear() throws HyracksDataException {
+        if (!isActivated) {
+            throw new HyracksDataException("Failed to clear the index since it is not activated.");
+        }
+
+        mutableComponent.getRTree().clear();
+        mutableComponent.getBTree().clear();
+    }
+
+    @Override
+    public void getOperationalComponents(ILSMIndexOperationContext ctx) {
+        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
+        operationalComponents.clear();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        switch (ctx.getOperation()) {
+            case INSERT:
+            case DELETE:
+            case FLUSH:
+                operationalComponents.add(mutableComponent);
+                break;
+            case SEARCH:
+                operationalComponents.add(mutableComponent);
+                operationalComponents.addAll(immutableComponents);
+                break;
+            case MERGE:
+                operationalComponents.addAll(immutableComponents);
+                break;
+            default:
+                throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
+        }
+    }
+
+    protected LSMComponentFileReferences getMergeTargetFileName(List<ILSMComponent> mergingDiskComponents)
+            throws HyracksDataException {
+        RTree firstTree = ((LSMRTreeImmutableComponent) mergingDiskComponents.get(0)).getRTree();
+        RTree lastTree = ((LSMRTreeImmutableComponent) mergingDiskComponents.get(mergingDiskComponents.size() - 1))
+                .getRTree();
+        FileReference firstFile = diskFileMapProvider.lookupFileName(firstTree.getFileId());
+        FileReference lastFile = diskFileMapProvider.lookupFileName(lastTree.getFileId());
+        LSMComponentFileReferences fileRefs = fileManager.getRelMergeFileReference(firstFile.getFile().getName(),
+                lastFile.getFile().getName());
+        return fileRefs;
+    }
+
+    protected LSMRTreeImmutableComponent createDiskComponent(ILSMComponentFactory factory, FileReference insertFileRef,
+            FileReference deleteFileRef, FileReference bloomFilterFileRef, boolean createComponent)
+            throws HyracksDataException, IndexException {
+        // Create new tree instance.
+        LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) factory
+                .createLSMComponentInstance(new LSMComponentFileReferences(insertFileRef, deleteFileRef,
+                        bloomFilterFileRef));
+        if (createComponent) {
+            component.getRTree().create();
+            if (component.getBTree() != null) {
+                component.getBTree().create();
+                component.getBloomFilter().create();
+            }
+        }
+        // Tree will be closed during cleanup of merge().
+        component.getRTree().activate();
+        if (component.getBTree() != null) {
+            component.getBTree().activate();
+            component.getBloomFilter().activate();
+        }
+        return component;
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getLeafFrameFactory() {
+        return mutableComponent.getRTree().getLeafFrameFactory();
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getInteriorFrameFactory() {
+        return mutableComponent.getRTree().getInteriorFrameFactory();
+    }
+
+    @Override
+    public IFreePageManager getFreePageManager() {
+        return mutableComponent.getRTree().getFreePageManager();
+    }
+
+    @Override
+    public int getFieldCount() {
+        return mutableComponent.getRTree().getFieldCount();
+    }
+
+    @Override
+    public int getRootPageId() {
+        return mutableComponent.getRTree().getRootPageId();
+    }
+
+    @Override
+    public int getFileId() {
+        return mutableComponent.getRTree().getFileId();
+    }
+
+    @Override
+    public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException, IndexException {
+        LSMRTreeOpContext ctx = (LSMRTreeOpContext) ictx;
+        if (ctx.getOperation() == IndexOperation.PHYSICALDELETE) {
+            throw new UnsupportedOperationException("Physical delete not supported in the LSM-RTree");
+        }
+
+        if (ctx.getOperation() == IndexOperation.INSERT) {
+            // Before each insert, we must check whether there exist a killer
+            // tuple in the memBTree. If we find a killer tuple, we must truly
+            // delete the existing tuple from the BTree, and then insert it to
+            // memRTree. Otherwise, the old killer tuple will kill the newly
+            // added RTree tuple.
+            RangePredicate btreeRangePredicate = new RangePredicate(tuple, tuple, true, true,
+                    ctx.getBTreeMultiComparator(), ctx.getBTreeMultiComparator());
+            ITreeIndexCursor cursor = ctx.memBTreeAccessor.createSearchCursor();
+            ctx.memBTreeAccessor.search(cursor, btreeRangePredicate);
+            boolean foundTupleInMemoryBTree = false;
+            try {
+                if (cursor.hasNext()) {
+                    foundTupleInMemoryBTree = true;
+                }
+            } finally {
+                cursor.close();
+            }
+            if (foundTupleInMemoryBTree) {
+                try {
+                    ctx.memBTreeAccessor.delete(tuple);
+                } catch (BTreeNonExistentKeyException e) {
+                    // Tuple has been deleted in the meantime. Do nothing.
+                    // This normally shouldn't happen if we are dealing with
+                    // good citizens since LSMRTree is used as a secondary
+                    // index and a tuple shouldn't be deleted twice without
+                    // insert between them.
+                }
+            } else {
+                ctx.memRTreeAccessor.insert(tuple);
+            }
+
+        } else {
+            try {
+                ctx.memBTreeAccessor.insert(tuple);
+            } catch (BTreeDuplicateKeyException e) {
+                // Do nothing, because one delete tuple is enough to indicate
+                // that all the corresponding insert tuples are deleted
+            }
+        }
+    }
+
+    protected LSMRTreeOpContext createOpContext(IModificationOperationCallback modCallback) {
+        return new LSMRTreeOpContext((RTree.RTreeAccessor) mutableComponent.getRTree().createAccessor(modCallback,
+                NoOpOperationCallback.INSTANCE), (IRTreeLeafFrame) rtreeLeafFrameFactory.createFrame(),
+                (IRTreeInteriorFrame) rtreeInteriorFrameFactory.createFrame(), memFreePageManager
+                        .getMetaDataFrameFactory().createFrame(), 4, (BTree.BTreeAccessor) mutableComponent.getBTree()
+                        .createAccessor(modCallback, NoOpOperationCallback.INSTANCE), btreeLeafFrameFactory,
+                btreeInteriorFrameFactory, memFreePageManager.getMetaDataFrameFactory().createFrame(),
+                rtreeCmpFactories, btreeCmpFactories, null, null);
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getComparatorFactories() {
+        return rtreeCmpFactories;
+    }
+
+    public boolean isEmptyIndex() throws HyracksDataException {
+        return componentsRef.get().isEmpty()
+                && mutableComponent.getBTree().isEmptyTree(
+                        mutableComponent.getBTree().getInteriorFrameFactory().createFrame())
+                && mutableComponent.getRTree().isEmptyTree(
+                        mutableComponent.getRTree().getInteriorFrameFactory().createFrame());
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        throw new UnsupportedOperationException("Validation not implemented for LSM R-Trees.");
+    }
+
+    @Override
+    public long getMemoryAllocationSize() {
+        InMemoryBufferCache memBufferCache = (InMemoryBufferCache) mutableComponent.getRTree().getBufferCache();
+        return memBufferCache.getNumPages() * memBufferCache.getPageSize();
+    }
+}
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
new file mode 100644
index 0000000..3bffb43
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -0,0 +1,474 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.util.List;
+import java.util.ListIterator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMRTree extends AbstractLSMRTree {
+
+    public LSMRTree(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
+            ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory rtreeLeafFrameFactory,
+            ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
+            ILSMIndexFileManager fileNameManager, TreeIndexFactory<RTree> diskRTreeFactory,
+            TreeIndexFactory<BTree> diskBTreeFactory, BloomFilterFactory bloomFilterFactory,
+            IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
+            int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory,
+                new LSMRTreeComponentFactory(diskRTreeFactory, diskBTreeFactory, bloomFilterFactory),
+                diskFileMapProvider, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
+                linearizerArray, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+    }
+
+    /**
+     * Opens LSMRTree, cleaning up invalid files from base dir, and registering
+     * all valid files as on-disk RTrees and BTrees.
+     * 
+     * @param fileReference
+     *            Dummy file id.
+     * @throws HyracksDataException
+     */
+    @Override
+    public synchronized void activate() throws HyracksDataException {
+        super.activate();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        List<LSMComponentFileReferences> validFileReferences;
+        try {
+            validFileReferences = fileManager.cleanupAndGetValidFiles();
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+        immutableComponents.clear();
+        for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
+            LSMRTreeImmutableComponent component;
+            try {
+                component = createDiskComponent(componentFactory,
+                        lsmComonentFileReference.getInsertIndexFileReference(),
+                        lsmComonentFileReference.getDeleteIndexFileReference(),
+                        lsmComonentFileReference.getBloomFilterFileReference(), false);
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+            immutableComponents.add(component);
+        }
+        isActivated = true;
+    }
+
+    @Override
+    public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
+        super.deactivate(flushOnExit);
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) c;
+            RTree rtree = component.getRTree();
+            BTree btree = component.getBTree();
+            BloomFilter bloomFilter = component.getBloomFilter();
+            rtree.deactivate();
+            btree.deactivate();
+            bloomFilter.deactivate();
+        }
+        isActivated = false;
+    }
+
+    @Override
+    public synchronized void deactivate() throws HyracksDataException {
+        deactivate(true);
+    }
+
+    @Override
+    public synchronized void destroy() throws HyracksDataException {
+        super.destroy();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) c;
+            component.getBTree().destroy();
+            component.getBloomFilter().destroy();
+            component.getRTree().destroy();
+        }
+        fileManager.deleteDirs();
+    }
+
+    @Override
+    public synchronized void clear() throws HyracksDataException {
+        super.clear();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) c;
+            component.getBTree().deactivate();
+            component.getBloomFilter().deactivate();
+            component.getRTree().deactivate();
+            component.getBTree().destroy();
+            component.getBloomFilter().destroy();
+            component.getRTree().destroy();
+        }
+        immutableComponents.clear();
+    }
+
+    @Override
+    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException {
+        LSMRTreeOpContext ctx = (LSMRTreeOpContext) ictx;
+        List<ILSMComponent> operationalComponents = ctx.getComponentHolder();
+        boolean includeMutableComponent = operationalComponents.get(0) == mutableComponent;
+        int numTrees = operationalComponents.size();
+
+        ListIterator<ILSMComponent> diskComponentIter = operationalComponents.listIterator();
+        ITreeIndexAccessor[] rTreeAccessors = new ITreeIndexAccessor[numTrees];
+        ITreeIndexAccessor[] bTreeAccessors = new ITreeIndexAccessor[numTrees];
+        int diskComponentIx = 0;
+        if (includeMutableComponent) {
+            rTreeAccessors[0] = ctx.memRTreeAccessor;
+            bTreeAccessors[0] = ctx.memBTreeAccessor;
+            diskComponentIx++;
+            diskComponentIter.next();
+        }
+
+        while (diskComponentIter.hasNext()) {
+            LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) diskComponentIter.next();
+            RTree diskRTree = component.getRTree();
+            BTree diskBTree = component.getBTree();
+            rTreeAccessors[diskComponentIx] = diskRTree.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            bTreeAccessors[diskComponentIx] = diskBTree.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            diskComponentIx++;
+        }
+
+        LSMRTreeCursorInitialState initialState = new LSMRTreeCursorInitialState(numTrees, rtreeLeafFrameFactory,
+                rtreeInteriorFrameFactory, btreeLeafFrameFactory, ctx.getBTreeMultiComparator(), rTreeAccessors,
+                bTreeAccessors, includeMutableComponent, lsmHarness, comparatorFields, linearizerArray,
+                ctx.searchCallback, operationalComponents);
+        cursor.open(initialState, pred);
+    }
+
+    @Override
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException {
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE);
+        LSMRTreeMutableComponent flushingComponent = (LSMRTreeMutableComponent) ctx.getComponentHolder().get(0);
+        rctx.setOperation(IndexOperation.FLUSH);
+        rctx.getComponentHolder().addAll(ctx.getComponentHolder());
+        LSMRTreeAccessor accessor = new LSMRTreeAccessor(lsmHarness, rctx);
+        ioScheduler.scheduleOperation(new LSMRTreeFlushOperation(accessor, flushingComponent, componentFileRefs
+                .getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(), componentFileRefs
+                .getBloomFilterFileReference(), callback));
+    }
+
+    @Override
+    public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        LSMRTreeFlushOperation flushOp = (LSMRTreeFlushOperation) operation;
+        LSMRTreeMutableComponent flushingComponent = (LSMRTreeMutableComponent) flushOp.getFlushingComponent();
+        // Renaming order is critical because we use assume ordering when we
+        // read the file names when we open the tree.
+        // The RTree should be renamed before the BTree.
+
+        // scan the memory RTree
+        ITreeIndexAccessor memRTreeAccessor = flushingComponent.getRTree().createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor();
+        SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
+        memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
+        LSMRTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(),
+                flushOp.getBTreeFlushTarget(), flushOp.getBloomFilterFlushTarget(), true);
+        RTree diskRTree = component.getRTree();
+        IIndexBulkLoader rTreeBulkloader;
+        ITreeIndexCursor cursor;
+
+        IBinaryComparatorFactory[] linearizerArray = { linearizer };
+
+        if (rTreeTupleSorter == null) {
+            rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getRTree().getFileId(), linearizerArray,
+                    rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(), flushingComponent
+                            .getRTree().getBufferCache(), comparatorFields);
+        } else {
+            rTreeTupleSorter.reset();
+        }
+        // BulkLoad the tuples from the in-memory tree into the new disk
+        // RTree.
+
+        boolean isEmpty = true;
+        try {
+            while (rtreeScanCursor.hasNext()) {
+                isEmpty = false;
+                rtreeScanCursor.next();
+                rTreeTupleSorter.insertTupleEntry(rtreeScanCursor.getPageId(), rtreeScanCursor.getTupleOffset());
+            }
+        } finally {
+            rtreeScanCursor.close();
+        }
+        if (!isEmpty) {
+            rTreeTupleSorter.sort();
+
+            rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L);
+            cursor = rTreeTupleSorter;
+
+            try {
+                while (cursor.hasNext()) {
+                    cursor.next();
+                    ITupleReference frameTuple = cursor.getTuple();
+                    rTreeBulkloader.add(frameTuple);
+                }
+            } finally {
+                cursor.close();
+            }
+            rTreeBulkloader.end();
+        }
+
+        ITreeIndexAccessor memBTreeAccessor = flushingComponent.getBTree().createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null);
+        IIndexCursor btreeCountingCursor = ((BTreeAccessor) memBTreeAccessor).createCountingSearchCursor();
+        memBTreeAccessor.search(btreeCountingCursor, btreeNullPredicate);
+        long numBTreeTuples = 0L;
+        try {
+            while (btreeCountingCursor.hasNext()) {
+                btreeCountingCursor.next();
+                ITupleReference countTuple = btreeCountingCursor.getTuple();
+                numBTreeTuples = IntegerSerializerDeserializer.getInt(countTuple.getFieldData(0),
+                        countTuple.getFieldStart(0));
+            }
+        } finally {
+            btreeCountingCursor.close();
+        }
+
+        if (numBTreeTuples > 0) {
+            int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numBTreeTuples);
+            BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                    MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+
+            IIndexCursor btreeScanCursor = memBTreeAccessor.createSearchCursor();
+            memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
+            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 builder = component.getBloomFilter().createBuilder(numBTreeTuples,
+                    bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+            // scan the memory BTree
+            try {
+                while (btreeScanCursor.hasNext()) {
+                    btreeScanCursor.next();
+                    ITupleReference frameTuple = btreeScanCursor.getTuple();
+                    bTreeBulkloader.add(frameTuple);
+                    builder.add(frameTuple);
+                }
+            } finally {
+                btreeScanCursor.close();
+                builder.end();
+            }
+            bTreeBulkloader.end();
+        }
+
+        return component;
+    }
+
+    @Override
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        // Renaming order is critical because we use assume ordering when we
+        // read the file names when we open the tree.
+        // The RTree should be renamed before the BTree.
+        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+        ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE);
+        rctx.getComponentHolder().addAll(mergingComponents);
+        ITreeIndexCursor cursor = new LSMRTreeSortedCursor(rctx, linearizer);
+        ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null);
+        search(rctx, cursor, rtreeSearchPred);
+
+        rctx.setOperation(IndexOperation.MERGE);
+        LSMComponentFileReferences relMergeFileRefs = getMergeTargetFileName(mergingComponents);
+        ILSMIndexAccessorInternal accessor = new LSMRTreeAccessor(lsmHarness, rctx);
+        ioScheduler.scheduleOperation(new LSMRTreeMergeOperation((ILSMIndexAccessorInternal) accessor,
+                mergingComponents, cursor, relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs
+                        .getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback));
+    }
+
+    @Override
+    public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
+            throws HyracksDataException, IndexException {
+        LSMRTreeMergeOperation mergeOp = (LSMRTreeMergeOperation) operation;
+        ITreeIndexCursor cursor = mergeOp.getCursor();
+        mergedComponents.addAll(mergeOp.getMergingComponents());
+
+        LSMRTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
+                mergeOp.getRTreeMergeTarget(), mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
+        IIndexBulkLoader bulkLoader = mergedComponent.getRTree().createBulkLoader(1.0f, false, 0L);
+
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+                ITupleReference frameTuple = cursor.getTuple();
+                bulkLoader.add(frameTuple);
+            }
+        } finally {
+            cursor.close();
+        }
+        bulkLoader.end();
+        return mergedComponent;
+    }
+
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new LSMRTreeAccessor(lsmHarness, createOpContext(modificationCallback));
+    }
+
+    public class LSMRTreeAccessor extends LSMTreeIndexAccessor {
+        public LSMRTreeAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx) {
+            super(lsmHarness, ctx);
+        }
+
+        @Override
+        public ITreeIndexCursor createSearchCursor() {
+            return new LSMRTreeSearchCursor(ctx);
+        }
+
+        public MultiComparator getMultiComparator() {
+            LSMRTreeOpContext concreteCtx = (LSMRTreeOpContext) ctx;
+            return concreteCtx.rtreeOpContext.cmp;
+        }
+    }
+
+    private ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
+        LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+        return createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
+                componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), true);
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
+            throws TreeIndexException {
+        return new LSMRTreeBulkLoader(fillLevel, verifyInput, numElementsHint);
+    }
+
+    public class LSMRTreeBulkLoader implements IIndexBulkLoader {
+        private final ILSMComponent component;
+        private final IIndexBulkLoader bulkLoader;
+
+        public LSMRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+                throws TreeIndexException {
+            // 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);
+            }
+            bulkLoader = ((LSMRTreeImmutableComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
+                    numElementsHint);
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws HyracksDataException, IndexException {
+            try {
+                bulkLoader.add(tuple);
+            } catch (IndexException e) {
+                handleException();
+                throw e;
+            } catch (HyracksDataException e) {
+                handleException();
+                throw e;
+            } catch (RuntimeException e) {
+                handleException();
+                throw e;
+            }
+        }
+
+        @Override
+        public void end() throws HyracksDataException, IndexException {
+            bulkLoader.end();
+            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();
+        }
+    }
+
+    @Override
+    public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException {
+        LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) lsmComponent;
+        // Flush the bloom filter first.
+        int fileId = component.getBloomFilter().getFileId();
+        IBufferCache bufferCache = component.getBTree().getBufferCache();
+        int startPage = 0;
+        int maxPage = component.getBloomFilter().getNumPages();
+        forceFlushDirtyPages(bufferCache, fileId, startPage, maxPage);
+        forceFlushDirtyPages(component.getRTree());
+        markAsValidInternal(component.getRTree());
+        forceFlushDirtyPages(component.getBTree());
+        markAsValidInternal(component.getBTree());
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
new file mode 100644
index 0000000..5a72f29
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -0,0 +1,143 @@
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;

+

+import java.util.List;

+

+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;

+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;

+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;

+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;

+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;

+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;

+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;

+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;

+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;

+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;

+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;

+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;

+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;

+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;

+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;

+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;

+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;

+import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;

+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;

+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;

+

+public abstract class LSMRTreeAbstractCursor implements ITreeIndexCursor {

+

+    protected RTreeSearchCursor[] rtreeCursors;

+    protected boolean open = false;

+    protected ITreeIndexCursor[] btreeCursors;

+    protected ITreeIndexAccessor[] rTreeAccessors;

+    protected ITreeIndexAccessor[] bTreeAccessors;

+    private MultiComparator btreeCmp;

+    protected int numberOfTrees;

+    protected SearchPredicate rtreeSearchPredicate;

+    protected RangePredicate btreeRangePredicate;

+    protected ITupleReference frameTuple;

+    protected boolean includeMemRTree;

+    protected ILSMHarness lsmHarness;

+    protected boolean foundNext;

+    protected final ILSMIndexOperationContext opCtx;

+

+    protected List<ILSMComponent> operationalComponents;

+

+    public LSMRTreeAbstractCursor(ILSMIndexOperationContext opCtx) {

+        super();

+        this.opCtx = opCtx;

+    }

+

+    public RTreeSearchCursor getCursor(int cursorIndex) {

+        return rtreeCursors[cursorIndex];

+    }

+

+    @Override

+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {

+        LSMRTreeCursorInitialState lsmInitialState = (LSMRTreeCursorInitialState) initialState;

+        btreeCmp = lsmInitialState.getBTreeCmp();

+        includeMemRTree = lsmInitialState.getIncludeMemComponent();

+        operationalComponents = lsmInitialState.getOperationalComponents();

+        lsmHarness = lsmInitialState.getLSMHarness();

+        numberOfTrees = lsmInitialState.getNumberOfTrees();

+        rTreeAccessors = lsmInitialState.getRTreeAccessors();

+        bTreeAccessors = lsmInitialState.getBTreeAccessors();

+

+        rtreeCursors = new RTreeSearchCursor[numberOfTrees];

+        btreeCursors = new ITreeIndexCursor[numberOfTrees];

+

+        int i = 0;

+        if (includeMemRTree) {

+            rtreeCursors[i] = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState

+                    .getRTreeInteriorFrameFactory().createFrame(), (IRTreeLeafFrame) lsmInitialState

+                    .getRTreeLeafFrameFactory().createFrame());

+

+            // No need for a bloom filter for the in-memory BTree.

+            btreeCursors[i] = new BTreeRangeSearchCursor((IBTreeLeafFrame) lsmInitialState.getBTreeLeafFrameFactory()

+                    .createFrame(), false);

+            ++i;

+        }

+        for (; i < numberOfTrees; i++) {

+            rtreeCursors[i] = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState

+                    .getRTreeInteriorFrameFactory().createFrame(), (IRTreeLeafFrame) lsmInitialState

+                    .getRTreeLeafFrameFactory().createFrame());

+

+            btreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitialState

+                    .getBTreeLeafFrameFactory().createFrame(), false,

+                    ((LSMRTreeImmutableComponent) operationalComponents.get(i)).getBloomFilter());

+        }

+

+        rtreeSearchPredicate = (SearchPredicate) searchPred;

+        btreeRangePredicate = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);

+

+        open = true;

+    }

+

+    @Override

+    public ICachedPage getPage() {

+        // do nothing

+        return null;

+    }

+

+    @Override

+    public void close() throws HyracksDataException {

+        if (!open) {

+            return;

+        }

+

+        try {

+            if (rtreeCursors != null && btreeCursors != null) {

+                for (int i = 0; i < numberOfTrees; i++) {

+                    rtreeCursors[i].close();

+                    btreeCursors[i].close();

+                }

+            }

+            rtreeCursors = null;

+            btreeCursors = null;

+        } finally {

+            lsmHarness.endSearch(opCtx);

+        }

+

+        open = false;

+    }

+

+    @Override

+    public void setBufferCache(IBufferCache bufferCache) {

+        // do nothing

+    }

+

+    @Override

+    public void setFileId(int fileId) {

+        // do nothing

+    }

+

+    @Override

+    public ITupleReference getTuple() {

+        return frameTuple;

+    }

+

+    @Override

+    public boolean exclusiveLatchNodes() {

+        return false;

+    }

+

+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeComponentFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeComponentFactory.java
new file mode 100644
index 0000000..56e3d28
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeComponentFactory.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public class LSMRTreeComponentFactory implements ILSMComponentFactory {
+    private final TreeIndexFactory<RTree> rtreeFactory;
+    private final TreeIndexFactory<BTree> btreeFactory;
+    private final BloomFilterFactory bloomFilterFactory;
+
+    public LSMRTreeComponentFactory(TreeIndexFactory<RTree> rtreeFactory, TreeIndexFactory<BTree> btreeFactory,
+            BloomFilterFactory bloomFilterFactory) {
+        this.rtreeFactory = rtreeFactory;
+        this.btreeFactory = btreeFactory;
+        this.bloomFilterFactory = bloomFilterFactory;
+    }
+
+    @Override
+    public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) throws IndexException,
+            HyracksDataException {
+        return new LSMRTreeImmutableComponent(rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+                btreeFactory.createIndexInstance(cfr.getDeleteIndexFileReference()),
+                bloomFilterFactory.createBloomFiltertInstance(cfr.getBloomFilterFileReference()));
+    }
+
+    @Override
+    public IBufferCache getBufferCache() {
+        return rtreeFactory.getBufferCache();
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeCursorInitialState.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeCursorInitialState.java
new file mode 100644
index 0000000..590d5d8
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeCursorInitialState.java
@@ -0,0 +1,144 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public class LSMRTreeCursorInitialState implements ICursorInitialState {
+
+    private final int numberOfTrees;
+    private final ITreeIndexFrameFactory rtreeInteriorFrameFactory;
+    private final ITreeIndexFrameFactory rtreeLeafFrameFactory;
+    private final ITreeIndexFrameFactory btreeLeafFrameFactory;
+    private final MultiComparator btreeCmp;
+    private final MultiComparator hilbertCmp;
+    private final ITreeIndexAccessor[] rTreeAccessors;
+    private final ITreeIndexAccessor[] bTreeAccessors;
+    private final boolean includeMemRTree;
+    private final ILSMHarness lsmHarness;
+    private final int[] comparatorFields;
+
+    private ISearchOperationCallback searchCallback;
+    private final List<ILSMComponent> operationalComponents;
+
+    public LSMRTreeCursorInitialState(int numberOfTrees, ITreeIndexFrameFactory rtreeLeafFrameFactory,
+            ITreeIndexFrameFactory rtreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
+            MultiComparator btreeCmp, ITreeIndexAccessor[] rTreeAccessors, ITreeIndexAccessor[] bTreeAccessors,
+            boolean includeMemRTree, ILSMHarness lsmHarness, int[] comparatorFields,
+            IBinaryComparatorFactory[] linearizerArray, ISearchOperationCallback searchCallback,
+            List<ILSMComponent> operationalComponents) {
+        this.numberOfTrees = numberOfTrees;
+        this.rtreeLeafFrameFactory = rtreeLeafFrameFactory;
+        this.rtreeInteriorFrameFactory = rtreeInteriorFrameFactory;
+        this.btreeLeafFrameFactory = btreeLeafFrameFactory;
+        this.btreeCmp = btreeCmp;
+        this.rTreeAccessors = rTreeAccessors;
+        this.bTreeAccessors = bTreeAccessors;
+        this.includeMemRTree = includeMemRTree;
+        this.lsmHarness = lsmHarness;
+        this.comparatorFields = comparatorFields;
+        this.hilbertCmp = MultiComparator.create(linearizerArray);
+        this.searchCallback = searchCallback;
+        this.operationalComponents = operationalComponents;
+    }
+
+    public MultiComparator getHilbertCmp() {
+        return hilbertCmp;
+    }
+
+    public int[] getComparatorFields() {
+        return comparatorFields;
+    }
+
+    public int getNumberOfTrees() {
+        return numberOfTrees;
+    }
+
+    public ITreeIndexFrameFactory getRTreeInteriorFrameFactory() {
+        return rtreeInteriorFrameFactory;
+    }
+
+    public ITreeIndexFrameFactory getRTreeLeafFrameFactory() {
+        return rtreeLeafFrameFactory;
+    }
+
+    public ITreeIndexFrameFactory getBTreeLeafFrameFactory() {
+        return btreeLeafFrameFactory;
+    }
+
+    public MultiComparator getBTreeCmp() {
+        return btreeCmp;
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        return null;
+    }
+
+    @Override
+    public void setPage(ICachedPage page) {
+    }
+
+    public List<ILSMComponent> getOperationalComponents() {
+        return operationalComponents;
+    }
+
+    public ITreeIndexAccessor[] getRTreeAccessors() {
+        return rTreeAccessors;
+    }
+
+    public ITreeIndexAccessor[] getBTreeAccessors() {
+        return bTreeAccessors;
+    }
+
+    public boolean getIncludeMemComponent() {
+        return includeMemRTree;
+    }
+
+    public ILSMHarness getLSMHarness() {
+        return lsmHarness;
+    }
+
+    @Override
+    public ISearchOperationCallback getSearchOperationCallback() {
+        return searchCallback;
+    }
+
+    @Override
+    public void setSearchOperationCallback(ISearchOperationCallback searchCallback) {
+        this.searchCallback = searchCallback;
+    }
+
+    @Override
+    public MultiComparator getOriginalKeyComparator() {
+        return null;
+    }
+
+    @Override
+    public void setOriginialKeyComparator(MultiComparator originalCmp) {
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
new file mode 100644
index 0000000..e698990
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
@@ -0,0 +1,229 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.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;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMRTreeFileManager extends AbstractLSMIndexFileManager {
+    private static final String RTREE_STRING = "r";
+    private static final String BTREE_STRING = "b";
+
+    private final TreeIndexFactory<? extends ITreeIndex> rtreeFactory;
+    private final TreeIndexFactory<? extends ITreeIndex> btreeFactory;
+
+    private static FilenameFilter btreeFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".") && name.endsWith(BTREE_STRING);
+        }
+    };
+
+    private static FilenameFilter rtreeFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".") && name.endsWith(RTREE_STRING);
+        }
+    };
+
+    public LSMRTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> rtreeFactory, TreeIndexFactory<? extends ITreeIndex> btreeFactory,
+            int startIODeviceIndex) {
+        super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+        this.rtreeFactory = rtreeFactory;
+        this.btreeFactory = btreeFactory;
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelFlushFileReference() {
+        Date date = new Date();
+        String ts = formatter.format(date);
+        String baseName = baseDir + ts + SPLIT_STRING + ts;
+        // Begin timestamp and end timestamp are identical since it is a flush
+        return new LSMComponentFileReferences(createFlushFile(baseName + SPLIT_STRING + RTREE_STRING),
+                createFlushFile(baseName + SPLIT_STRING + BTREE_STRING), createFlushFile(baseName + SPLIT_STRING
+                        + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
+            throws HyracksDataException {
+        String[] firstTimestampRange = firstFileName.split(SPLIT_STRING);
+        String[] lastTimestampRange = lastFileName.split(SPLIT_STRING);
+
+        String baseName = baseDir + firstTimestampRange[0] + SPLIT_STRING + lastTimestampRange[1];
+        // Get the range of timestamps by taking the earliest and the latest timestamps
+        return new LSMComponentFileReferences(createMergeFile(baseName + SPLIT_STRING + RTREE_STRING),
+                createMergeFile(baseName + SPLIT_STRING + BTREE_STRING), createMergeFile(baseName + SPLIT_STRING
+                        + BLOOM_FILTER_STRING));
+    }
+
+    @Override
+    public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException, IndexException {
+        List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
+        ArrayList<ComparableFileName> allRTreeFiles = new ArrayList<ComparableFileName>();
+        ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
+        ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
+
+        // Gather files from all IODeviceHandles.
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            cleanupAndGetValidFilesInternal(dev, bloomFilterFilter, null, allBloomFilterFiles);
+            HashSet<String> bloomFilterFilesSet = new HashSet<String>();
+            for (ComparableFileName cmpFileName : allBloomFilterFiles) {
+                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+                bloomFilterFilesSet.add(cmpFileName.fileName.substring(0, index));
+            }
+
+            // List of valid BTree files that may or may not have a bloom filter buddy. Will check for buddies below.
+            ArrayList<ComparableFileName> tmpAllBTreeFiles = new ArrayList<ComparableFileName>();
+            cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, tmpAllBTreeFiles);
+            // Look for buddy bloom filters for all valid BTrees. 
+            // If no buddy is found, delete the file, otherwise add the BTree to allBTreeFiles. 
+            HashSet<String> btreeFilesSet = new HashSet<String>();
+            for (ComparableFileName cmpFileName : tmpAllBTreeFiles) {
+                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+                String file = cmpFileName.fileName.substring(0, index);
+                if (bloomFilterFilesSet.contains(file)) {
+                    allBTreeFiles.add(cmpFileName);
+                    btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
+                } else {
+                    // Couldn't find the corresponding bloom filter file; thus, delete
+                    // the BTree file.
+                    File invalidBTreeFile = new File(cmpFileName.fullPath);
+                    invalidBTreeFile.delete();
+                }
+            }
+
+            // List of valid RTree files that may or may not have a BTree buddy. Will check for buddies below.
+            ArrayList<ComparableFileName> tmpAllRTreeFiles = new ArrayList<ComparableFileName>();
+            cleanupAndGetValidFilesInternal(dev, rtreeFilter, rtreeFactory, tmpAllRTreeFiles);
+            // Look for buddy BTrees for all valid RTrees. 
+            // If no buddy is found, delete the file, otherwise add the RTree to allRTreeFiles. 
+            for (ComparableFileName cmpFileName : tmpAllRTreeFiles) {
+                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+                String file = cmpFileName.fileName.substring(0, index);
+                if (btreeFilesSet.contains(file)) {
+                    allRTreeFiles.add(cmpFileName);
+                } else {
+                    // Couldn't find the corresponding BTree file; thus, delete
+                    // the RTree file.
+                    File invalidRTreeFile = new File(cmpFileName.fullPath);
+                    invalidRTreeFile.delete();
+                }
+            }
+        }
+        // Sanity check.
+        if (allRTreeFiles.size() != allBTreeFiles.size() || allBTreeFiles.size() != allBloomFilterFiles.size()) {
+            throw new HyracksDataException(
+                    "Unequal number of valid RTree, BTree, and Bloom Filter files found. Aborting cleanup.");
+        }
+
+        // Trivial cases.
+        if (allRTreeFiles.isEmpty() || allBTreeFiles.isEmpty() || allBloomFilterFiles.isEmpty()) {
+            return validFiles;
+        }
+
+        if (allRTreeFiles.size() == 1 && allBTreeFiles.size() == 1 && allBloomFilterFiles.size() == 1) {
+            validFiles.add(new LSMComponentFileReferences(allRTreeFiles.get(0).fileRef, allBTreeFiles.get(0).fileRef,
+                    allBloomFilterFiles.get(0).fileRef));
+            return validFiles;
+        }
+
+        // Sorts files names from earliest to latest timestamp.
+        Collections.sort(allRTreeFiles);
+        Collections.sort(allBTreeFiles);
+        Collections.sort(allBloomFilterFiles);
+
+        List<ComparableFileName> validComparableRTreeFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastRTree = allRTreeFiles.get(0);
+        validComparableRTreeFiles.add(lastRTree);
+
+        List<ComparableFileName> validComparableBTreeFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastBTree = allBTreeFiles.get(0);
+        validComparableBTreeFiles.add(lastBTree);
+
+        List<ComparableFileName> validComparableBloomFilterFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName lastBloomFilter = allBloomFilterFiles.get(0);
+        validComparableBloomFilterFiles.add(lastBloomFilter);
+
+        for (int i = 1; i < allRTreeFiles.size(); i++) {
+            ComparableFileName currentRTree = allRTreeFiles.get(i);
+            ComparableFileName currentBTree = allBTreeFiles.get(i);
+            ComparableFileName currentBloomFilter = allBloomFilterFiles.get(i);
+            // Current start timestamp is greater than last stop timestamp.
+            if (currentRTree.interval[0].compareTo(lastRTree.interval[1]) > 0
+                    && currentBTree.interval[0].compareTo(lastBTree.interval[1]) > 0
+                    && currentBloomFilter.interval[0].compareTo(lastBloomFilter.interval[1]) > 0) {
+                validComparableRTreeFiles.add(currentRTree);
+                validComparableBTreeFiles.add(currentBTree);
+                validComparableBloomFilterFiles.add(currentBloomFilter);
+                lastRTree = currentRTree;
+                lastBTree = currentBTree;
+                lastBloomFilter = currentBloomFilter;
+            } else if (currentRTree.interval[0].compareTo(lastRTree.interval[0]) >= 0
+                    && currentRTree.interval[1].compareTo(lastRTree.interval[1]) <= 0
+                    && currentBTree.interval[0].compareTo(lastBTree.interval[0]) >= 0
+                    && currentBTree.interval[1].compareTo(lastBTree.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 invalidBloomFilterFile = new File(currentBloomFilter.fullPath);
+                invalidBloomFilterFile.delete();
+            } else {
+                // This scenario should not be possible.
+                throw new HyracksDataException("Found LSM files with overlapping but not contained timetamp intervals.");
+            }
+        }
+
+        // Sort valid files in reverse lexicographical order, such that newer
+        // files come first.
+        Collections.sort(validComparableRTreeFiles, recencyCmp);
+        Collections.sort(validComparableBTreeFiles, recencyCmp);
+        Collections.sort(validComparableBloomFilterFiles, recencyCmp);
+
+        Iterator<ComparableFileName> rtreeFileIter = validComparableRTreeFiles.iterator();
+        Iterator<ComparableFileName> btreeFileIter = validComparableBTreeFiles.iterator();
+        Iterator<ComparableFileName> bloomFilterFileIter = validComparableBloomFilterFiles.iterator();
+        while (rtreeFileIter.hasNext() && btreeFileIter.hasNext()) {
+            ComparableFileName cmpRTreeFileName = rtreeFileIter.next();
+            ComparableFileName cmpBTreeFileName = btreeFileIter.next();
+            ComparableFileName cmpBloomFilterFileName = bloomFilterFileIter.next();
+            validFiles.add(new LSMComponentFileReferences(cmpRTreeFileName.fileRef, cmpBTreeFileName.fileRef,
+                    cmpBloomFilterFileName.fileRef));
+        }
+
+        return validFiles;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java
new file mode 100644
index 0000000..7b7f2bc
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java
@@ -0,0 +1,77 @@
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+
+public class LSMRTreeFlushOperation implements ILSMIOOperation {
+
+    private final ILSMIndexAccessorInternal accessor;
+    private final ILSMComponent flushingComponent;
+    private final FileReference rtreeFlushTarget;
+    private final FileReference btreeFlushTarget;
+    private final FileReference bloomFilterFlushTarget;
+    private final ILSMIOOperationCallback callback;
+
+    public LSMRTreeFlushOperation(ILSMIndexAccessorInternal accessor, ILSMComponent flushingComponent,
+            FileReference rtreeFlushTarget, FileReference btreeFlushTarget, FileReference bloomFilterFlushTarget,
+            ILSMIOOperationCallback callback) {
+        this.accessor = accessor;
+        this.flushingComponent = flushingComponent;
+        this.rtreeFlushTarget = rtreeFlushTarget;
+        this.btreeFlushTarget = btreeFlushTarget;
+        this.bloomFilterFlushTarget = bloomFilterFlushTarget;
+        this.callback = callback;
+    }
+
+    @Override
+    public Set<IODeviceHandle> getReadDevices() {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public Set<IODeviceHandle> getWriteDevices() {
+        Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+        devs.add(rtreeFlushTarget.getDeviceHandle());
+        if (btreeFlushTarget != null) {
+            devs.add(btreeFlushTarget.getDeviceHandle());
+            devs.add(bloomFilterFlushTarget.getDeviceHandle());
+        }
+        return devs;
+    }
+
+    @Override
+    public void perform() throws HyracksDataException, IndexException {
+        accessor.flush(this);
+    }
+
+    @Override
+    public ILSMIOOperationCallback getCallback() {
+        return callback;
+    }
+
+    public FileReference getRTreeFlushTarget() {
+        return rtreeFlushTarget;
+    }
+
+    public FileReference getBTreeFlushTarget() {
+        return btreeFlushTarget;
+    }
+
+    public FileReference getBloomFilterFlushTarget() {
+        return bloomFilterFlushTarget;
+    }
+
+    public ILSMComponent getFlushingComponent() {
+        return flushingComponent;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeImmutableComponent.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeImmutableComponent.java
new file mode 100644
index 0000000..8d20c14
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeImmutableComponent.java
@@ -0,0 +1,43 @@
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractImmutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+
+public class LSMRTreeImmutableComponent extends AbstractImmutableLSMComponent {
+    private final RTree rtree;
+    private final BTree btree;
+    private final BloomFilter bloomFilter;
+
+    public LSMRTreeImmutableComponent(RTree rtree, BTree btree, BloomFilter bloomFilter) {
+        this.rtree = rtree;
+        this.btree = btree;
+        this.bloomFilter = bloomFilter;
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        rtree.deactivate();
+        rtree.destroy();
+        if (btree != null) {
+            btree.deactivate();
+            btree.destroy();
+            bloomFilter.deactivate();
+            bloomFilter.destroy();
+        }
+    }
+
+    public RTree getRTree() {
+        return rtree;
+    }
+
+    public BTree getBTree() {
+        return btree;
+    }
+
+    public BloomFilter getBloomFilter() {
+        return bloomFilter;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
new file mode 100644
index 0000000..0e05a93
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
@@ -0,0 +1,92 @@
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+
+public class LSMRTreeMergeOperation implements ILSMIOOperation {
+    private final ILSMIndexAccessorInternal accessor;
+    private final List<ILSMComponent> mergingComponents;
+    private final ITreeIndexCursor cursor;
+    private final FileReference rtreeMergeTarget;
+    private final FileReference btreeMergeTarget;
+    private final FileReference bloomFilterMergeTarget;
+    private final ILSMIOOperationCallback callback;
+
+    public LSMRTreeMergeOperation(ILSMIndexAccessorInternal accessor, List<ILSMComponent> mergingComponents,
+            ITreeIndexCursor cursor, FileReference rtreeMergeTarget, FileReference btreeMergeTarget,
+            FileReference bloomFilterMergeTarget, ILSMIOOperationCallback callback) {
+        this.accessor = accessor;
+        this.mergingComponents = mergingComponents;
+        this.cursor = cursor;
+        this.rtreeMergeTarget = rtreeMergeTarget;
+        this.btreeMergeTarget = btreeMergeTarget;
+        this.bloomFilterMergeTarget = bloomFilterMergeTarget;
+        this.callback = callback;
+    }
+
+    @Override
+    public Set<IODeviceHandle> getReadDevices() {
+        Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+        for (ILSMComponent o : mergingComponents) {
+            LSMRTreeImmutableComponent component = (LSMRTreeImmutableComponent) o;
+            devs.add(component.getRTree().getFileReference().getDeviceHandle());
+            if (component.getBTree() != null) {
+                devs.add(component.getBTree().getFileReference().getDeviceHandle());
+                devs.add(component.getBloomFilter().getFileReference().getDeviceHandle());
+            }
+        }
+        return devs;
+    }
+
+    @Override
+    public Set<IODeviceHandle> getWriteDevices() {
+        Set<IODeviceHandle> devs = new HashSet<IODeviceHandle>();
+        devs.add(rtreeMergeTarget.getDeviceHandle());
+        if (btreeMergeTarget != null) {
+            devs.add(btreeMergeTarget.getDeviceHandle());
+            devs.add(bloomFilterMergeTarget.getDeviceHandle());
+        }
+        return devs;
+    }
+
+    @Override
+    public void perform() throws HyracksDataException, IndexException {
+        accessor.merge(this);
+    }
+
+    @Override
+    public ILSMIOOperationCallback getCallback() {
+        return callback;
+    }
+
+    public FileReference getRTreeMergeTarget() {
+        return rtreeMergeTarget;
+    }
+
+    public FileReference getBTreeMergeTarget() {
+        return btreeMergeTarget;
+    }
+
+    public FileReference getBloomFilterMergeTarget() {
+        return bloomFilterMergeTarget;
+    }
+
+    public ITreeIndexCursor getCursor() {
+        return cursor;
+    }
+
+    public List<ILSMComponent> getMergingComponents() {
+        return mergingComponents;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMutableComponent.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMutableComponent.java
new file mode 100644
index 0000000..80f76a1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMutableComponent.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractMutableLSMComponent;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+
+public class LSMRTreeMutableComponent extends AbstractMutableLSMComponent {
+
+    private final RTree rtree;
+    private final BTree btree;
+    private final IInMemoryFreePageManager mfpm;
+
+    public LSMRTreeMutableComponent(RTree rtree, BTree btree, IInMemoryFreePageManager mfpm) {
+        this.rtree = rtree;
+        this.btree = btree;
+        this.mfpm = mfpm;
+    }
+
+    public RTree getRTree() {
+        return rtree;
+    }
+
+    public BTree getBTree() {
+        return btree;
+    }
+
+    @Override
+    protected boolean isFull() {
+        return mfpm.isFull();
+    }
+
+    @Override
+    protected void reset() throws HyracksDataException {
+        rtree.clear();
+        if (btree != null) {
+            btree.clear();
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
new file mode 100644
index 0000000..b8805d1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
+
+public final class LSMRTreeOpContext implements ILSMIndexOperationContext {
+
+    public RTreeOpContext rtreeOpContext;
+    public BTreeOpContext btreeOpContext;
+    public final RTree.RTreeAccessor memRTreeAccessor;
+    public final BTree.BTreeAccessor memBTreeAccessor;
+    private IndexOperation op;
+    public final List<ILSMComponent> componentHolder;
+    public final IModificationOperationCallback modificationCallback;
+    public final ISearchOperationCallback searchCallback;
+
+    public LSMRTreeOpContext(RTree.RTreeAccessor memRtreeAccessor, IRTreeLeafFrame rtreeLeafFrame,
+            IRTreeInteriorFrame rtreeInteriorFrame, ITreeIndexMetaDataFrame rtreeMetaFrame, int rTreeHeightHint,
+            BTree.BTreeAccessor memBtreeAccessor, ITreeIndexFrameFactory btreeLeafFrameFactory,
+            ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexMetaDataFrame btreeMetaFrame,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback) {
+        this.memRTreeAccessor = memRtreeAccessor;
+        this.memBTreeAccessor = memBtreeAccessor;
+        this.componentHolder = new LinkedList<ILSMComponent>();
+        this.modificationCallback = modificationCallback;
+        this.searchCallback = searchCallback;
+        this.rtreeOpContext = new RTreeOpContext(rtreeLeafFrame, rtreeInteriorFrame, rtreeMetaFrame, rtreeCmpFactories,
+                rTreeHeightHint, NoOpOperationCallback.INSTANCE);
+        this.btreeOpContext = new BTreeOpContext(memBtreeAccessor, btreeLeafFrameFactory, btreeInteriorFrameFactory,
+                btreeMetaFrame, btreeCmpFactories, NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+    }
+
+    public void setOperation(IndexOperation newOp) {
+        reset();
+        if (newOp == IndexOperation.INSERT) {
+            rtreeOpContext.setOperation(newOp);
+        } else if (newOp == IndexOperation.DELETE) {
+            btreeOpContext.setOperation(IndexOperation.INSERT);
+        }
+        this.op = newOp;
+    }
+
+    @Override
+    public void reset() {
+        componentHolder.clear();
+    }
+
+    @Override
+    public IndexOperation getOperation() {
+        return op;
+    }
+
+    public MultiComparator getBTreeMultiComparator() {
+        return btreeOpContext.cmp;
+    }
+
+    @Override
+    public List<ILSMComponent> getComponentHolder() {
+        return componentHolder;
+    }
+
+    @Override
+    public ISearchOperationCallback getSearchOperationCallback() {
+        return searchCallback;
+    }
+
+    @Override
+    public IModificationOperationCallback getModificationCallback() {
+        return modificationCallback;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
new file mode 100644
index 0000000..966ed8d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
+
+    private int currentCursor;
+
+    public LSMRTreeSearchCursor(ILSMIndexOperationContext opCtx) {
+        super(opCtx);
+        currentCursor = 0;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        super.close();
+        currentCursor = 0;
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        if (!open) {
+            return;
+        }
+
+        currentCursor = 0;
+        foundNext = false;
+        try {
+            for (int i = 0; i < numberOfTrees; i++) {
+                rtreeCursors[i].close();
+                btreeCursors[i].close();
+            }
+            rtreeCursors = null;
+            btreeCursors = null;
+        } finally {
+            lsmHarness.endSearch(opCtx);
+        }
+    }
+
+    private void searchNextCursor() throws HyracksDataException {
+        if (currentCursor < numberOfTrees) {
+            rtreeCursors[currentCursor].reset();
+            try {
+                rTreeAccessors[currentCursor].search(rtreeCursors[currentCursor], rtreeSearchPredicate);
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        if (foundNext) {
+            return true;
+        }
+        while (currentCursor < numberOfTrees) {
+            while (rtreeCursors[currentCursor].hasNext()) {
+                rtreeCursors[currentCursor].next();
+                ITupleReference currentTuple = rtreeCursors[currentCursor].getTuple();
+
+                boolean killerTupleFound = false;
+                for (int i = 0; i <= currentCursor; i++) {
+                    try {
+                        btreeCursors[i].reset();
+                        btreeRangePredicate.setHighKey(currentTuple, true);
+                        btreeRangePredicate.setLowKey(currentTuple, true);
+                        bTreeAccessors[i].search(btreeCursors[i], btreeRangePredicate);
+                    } catch (IndexException e) {
+                        throw new HyracksDataException(e);
+                    }
+                    try {
+                        if (btreeCursors[i].hasNext()) {
+                            killerTupleFound = true;
+                            break;
+                        }
+                    } finally {
+                        btreeCursors[i].close();
+                    }
+                }
+                if (!killerTupleFound) {
+                    frameTuple = currentTuple;
+                    foundNext = true;
+                    return true;
+                }
+            }
+            rtreeCursors[currentCursor].close();
+            currentCursor++;
+            searchNextCursor();
+        }
+        return false;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        foundNext = false;
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        super.open(initialState, searchPred);
+        searchNextCursor();
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
new file mode 100644
index 0000000..02a1876
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
@@ -0,0 +1,152 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+
+public class LSMRTreeSortedCursor extends LSMRTreeAbstractCursor {
+
+    private ILinearizeComparator linearizeCmp;
+    private boolean[] depletedRtreeCursors;
+    private int foundIn = -1;
+
+    public LSMRTreeSortedCursor(ILSMIndexOperationContext opCtx, ILinearizeComparatorFactory linearizer)
+            throws HyracksDataException {
+        super(opCtx);
+        this.linearizeCmp = linearizer.createBinaryComparator();
+        reset();
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        depletedRtreeCursors = new boolean[numberOfTrees];
+        foundNext = false;
+        try {
+            for (int i = 0; i < numberOfTrees; i++) {
+                rtreeCursors[i].reset();
+                try {
+                    rTreeAccessors[i].search(rtreeCursors[i], rtreeSearchPredicate);
+                } catch (IndexException e) {
+                    throw new HyracksDataException(e);
+                }
+                if (rtreeCursors[i].hasNext()) {
+                    rtreeCursors[i].next();
+                } else {
+                    depletedRtreeCursors[i] = true;
+                }
+            }
+        } finally {
+            if (open) {
+                lsmHarness.endSearch(opCtx);
+            }
+        }
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        while (!foundNext) {
+            frameTuple = null;
+
+            if (foundIn != -1) {
+                if (rtreeCursors[foundIn].hasNext()) {
+                    rtreeCursors[foundIn].next();
+                } else {
+                    depletedRtreeCursors[foundIn] = true;
+                }
+            }
+
+            foundIn = -1;
+            for (int i = 0; i < numberOfTrees; i++) {
+                if (depletedRtreeCursors[i])
+                    continue;
+
+                if (frameTuple == null) {
+                    frameTuple = rtreeCursors[i].getTuple();
+                    foundIn = i;
+                    continue;
+                }
+
+                if (linearizeCmp.compare(frameTuple.getFieldData(0), frameTuple.getFieldStart(0),
+                        frameTuple.getFieldLength(0) * linearizeCmp.getDimensions(), rtreeCursors[i].getTuple()
+                                .getFieldData(0), rtreeCursors[i].getTuple().getFieldStart(0), rtreeCursors[i]
+                                .getTuple().getFieldLength(0) * linearizeCmp.getDimensions()) <= 0) {
+                    frameTuple = rtreeCursors[i].getTuple();
+                    foundIn = i;
+                }
+            }
+
+            if (foundIn == -1)
+                return false;
+
+            boolean killed = false;
+            for (int i = 0; i < foundIn; i++) {
+                try {
+                    btreeCursors[i].reset();
+                    btreeRangePredicate.setHighKey(frameTuple, true);
+                    btreeRangePredicate.setLowKey(frameTuple, true);
+                    bTreeAccessors[i].search(btreeCursors[i], btreeRangePredicate);
+                } catch (IndexException e) {
+                    throw new HyracksDataException(e);
+                }
+                try {
+                    if (btreeCursors[i].hasNext()) {
+                        killed = true;
+                        break;
+                    }
+                } finally {
+                    btreeCursors[i].close();
+                }
+            }
+            if (!killed) {
+                foundNext = true;
+            }
+        }
+
+        return true;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        foundNext = false;
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        super.open(initialState, searchPred);
+
+        depletedRtreeCursors = new boolean[numberOfTrees];
+        foundNext = false;
+        for (int i = 0; i < numberOfTrees; i++) {
+            rtreeCursors[i].reset();
+            try {
+                rTreeAccessors[i].search(rtreeCursors[i], rtreeSearchPredicate);
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+            if (rtreeCursors[i].hasNext()) {
+                rtreeCursors[i].next();
+            } else {
+                depletedRtreeCursors[i] = true;
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
new file mode 100644
index 0000000..478d076
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -0,0 +1,434 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.util.List;
+import java.util.ListIterator;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
+
+    private TreeTupleSorter bTreeTupleSorter;
+
+    // On-disk components.
+    // For creating RTree's used in bulk load. Different from diskRTreeFactory
+    // because it should have a different tuple writer in it's leaf frames.
+    private final ILSMComponentFactory bulkLoaComponentFactory;
+
+    public LSMRTreeWithAntiMatterTuples(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
+            ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory btreeInteriorFrameFactory,
+            ITreeIndexFrameFactory btreeLeafFrameFactory, ILSMIndexFileManager fileManager,
+            TreeIndexFactory<RTree> diskRTreeFactory, TreeIndexFactory<RTree> bulkLoadRTreeFactory,
+            IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
+            int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, fileManager, diskRTreeFactory,
+                new LSMRTreeWithAntiMatterTuplesComponentFactory(diskRTreeFactory), diskFileMapProvider, fieldCount,
+                rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        bulkLoaComponentFactory = new LSMRTreeWithAntiMatterTuplesComponentFactory(bulkLoadRTreeFactory);
+        this.bTreeTupleSorter = null;
+    }
+
+    @Override
+    public synchronized void activate() throws HyracksDataException {
+        super.activate();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        immutableComponents.clear();
+        List<LSMComponentFileReferences> validFileReferences;
+        try {
+            validFileReferences = fileManager.cleanupAndGetValidFiles();
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+        for (LSMComponentFileReferences lsmComonentFileReference : validFileReferences) {
+            LSMRTreeImmutableComponent component;
+            try {
+                component = createDiskComponent(componentFactory,
+                        lsmComonentFileReference.getInsertIndexFileReference(), null, null, false);
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
+            immutableComponents.add(component);
+        }
+        isActivated = true;
+    }
+
+    @Override
+    public synchronized void deactivate(boolean flushOnExit) throws HyracksDataException {
+        super.deactivate(flushOnExit);
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            RTree rtree = (RTree) ((LSMRTreeImmutableComponent) c).getRTree();
+            rtree.deactivate();
+        }
+        isActivated = false;
+    }
+
+    @Override
+    public synchronized void deactivate() throws HyracksDataException {
+        deactivate(true);
+    }
+
+    @Override
+    public synchronized void destroy() throws HyracksDataException {
+        super.destroy();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            RTree rtree = (RTree) ((LSMRTreeImmutableComponent) c).getRTree();
+            rtree.destroy();
+        }
+        fileManager.deleteDirs();
+    }
+
+    @Override
+    public synchronized void clear() throws HyracksDataException {
+        super.clear();
+        List<ILSMComponent> immutableComponents = componentsRef.get();
+        for (ILSMComponent c : immutableComponents) {
+            RTree rtree = (RTree) ((LSMRTreeImmutableComponent) c).getRTree();
+            rtree.deactivate();
+            rtree.destroy();
+        }
+        immutableComponents.clear();
+    }
+
+    @Override
+    public void search(ILSMIndexOperationContext ictx, IIndexCursor cursor, ISearchPredicate pred)
+            throws HyracksDataException, IndexException {
+        LSMRTreeOpContext ctx = (LSMRTreeOpContext) ictx;
+        List<ILSMComponent> operationalComponents = ictx.getComponentHolder();
+        boolean includeMutableComponent = operationalComponents.get(0) == mutableComponent;
+        LSMRTreeWithAntiMatterTuplesSearchCursor lsmTreeCursor = (LSMRTreeWithAntiMatterTuplesSearchCursor) cursor;
+        int numDiskRComponents = operationalComponents.size();
+
+        LSMRTreeCursorInitialState initialState;
+        ITreeIndexAccessor[] bTreeAccessors = null;
+        if (includeMutableComponent) {
+            // Only in-memory BTree
+            bTreeAccessors = new ITreeIndexAccessor[1];
+            bTreeAccessors[0] = ctx.memBTreeAccessor;
+        }
+
+        initialState = new LSMRTreeCursorInitialState(numDiskRComponents, rtreeLeafFrameFactory,
+                rtreeInteriorFrameFactory, btreeLeafFrameFactory, ctx.getBTreeMultiComparator(), null, bTreeAccessors,
+                includeMutableComponent, lsmHarness, comparatorFields, linearizerArray, ctx.searchCallback,
+                operationalComponents);
+
+        lsmTreeCursor.open(initialState, pred);
+
+        ListIterator<ILSMComponent> diskComponentsIter = operationalComponents.listIterator();
+        int diskComponentIx = 0;
+        if (includeMutableComponent) {
+            // Open cursor of in-memory RTree
+            ctx.memRTreeAccessor.search(lsmTreeCursor.getMemRTreeCursor(), pred);
+            diskComponentIx++;
+            diskComponentsIter.next();
+        }
+
+        // Open cursors of on-disk RTrees.
+        ITreeIndexAccessor[] diskRTreeAccessors = new ITreeIndexAccessor[numDiskRComponents];
+        while (diskComponentsIter.hasNext()) {
+            RTree diskRTree = (RTree) ((LSMRTreeImmutableComponent) diskComponentsIter.next()).getRTree();
+            diskRTreeAccessors[diskComponentIx] = diskRTree.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            diskRTreeAccessors[diskComponentIx].search(lsmTreeCursor.getCursor(diskComponentIx), pred);
+            diskComponentIx++;
+        }
+        lsmTreeCursor.initPriorityQueue();
+    }
+
+    @Override
+    public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException {
+        LSMRTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE);
+        LSMComponentFileReferences relFlushFileRefs = fileManager.getRelFlushFileReference();
+        ILSMComponent flushingComponent = ctx.getComponentHolder().get(0);
+        opCtx.setOperation(IndexOperation.FLUSH);
+        opCtx.getComponentHolder().add(flushingComponent);
+        ILSMIndexAccessorInternal accessor = new LSMRTreeWithAntiMatterTuplesAccessor(lsmHarness, opCtx);
+        ioScheduler.scheduleOperation(new LSMRTreeFlushOperation(accessor, flushingComponent, relFlushFileRefs
+                .getInsertIndexFileReference(), null, null, callback));
+    }
+
+    @Override
+    public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataException, IndexException {
+        LSMRTreeFlushOperation flushOp = (LSMRTreeFlushOperation) operation;
+        // Renaming order is critical because we use assume ordering when we
+        // read the file names when we open the tree.
+        // The RTree should be renamed before the BTree.
+        LSMRTreeMutableComponent flushingComponent = (LSMRTreeMutableComponent) flushOp.getFlushingComponent();
+        ITreeIndexAccessor memRTreeAccessor = flushingComponent.getRTree().createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor();
+        SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
+        memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
+        LSMRTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(),
+                null, null, true);
+        RTree diskRTree = component.getRTree();
+
+        // scan the memory BTree
+        ITreeIndexAccessor memBTreeAccessor = flushingComponent.getBTree().createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        BTreeRangeSearchCursor btreeScanCursor = (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor();
+        RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null);
+        memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
+
+        // Since the LSM-RTree is used as a secondary assumption, the
+        // primary key will be the last comparator in the BTree comparators
+        if (rTreeTupleSorter == null) {
+            rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getRTree().getFileId(), linearizerArray,
+                    rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(), flushingComponent
+                            .getRTree().getBufferCache(), comparatorFields);
+
+            bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBTree().getFileId(), linearizerArray,
+                    btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(), flushingComponent
+                            .getBTree().getBufferCache(), comparatorFields);
+        } else {
+            rTreeTupleSorter.reset();
+            bTreeTupleSorter.reset();
+        }
+        // BulkLoad the tuples from the in-memory tree into the new disk
+        // RTree.
+
+        boolean isEmpty = true;
+        try {
+            while (rtreeScanCursor.hasNext()) {
+                isEmpty = false;
+                rtreeScanCursor.next();
+                rTreeTupleSorter.insertTupleEntry(rtreeScanCursor.getPageId(), rtreeScanCursor.getTupleOffset());
+            }
+        } finally {
+            rtreeScanCursor.close();
+        }
+        if (!isEmpty) {
+            rTreeTupleSorter.sort();
+        }
+
+        isEmpty = true;
+        try {
+            while (btreeScanCursor.hasNext()) {
+                isEmpty = false;
+                btreeScanCursor.next();
+                bTreeTupleSorter.insertTupleEntry(btreeScanCursor.getPageId(), btreeScanCursor.getTupleOffset());
+            }
+        } finally {
+            btreeScanCursor.close();
+        }
+        if (!isEmpty) {
+            bTreeTupleSorter.sort();
+        }
+
+        IIndexBulkLoader rTreeBulkloader = diskRTree.createBulkLoader(1.0f, false, 0L);
+        LSMRTreeWithAntiMatterTuplesFlushCursor cursor = new LSMRTreeWithAntiMatterTuplesFlushCursor(rTreeTupleSorter,
+                bTreeTupleSorter, comparatorFields, linearizerArray);
+        cursor.open(null, null);
+
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+                ITupleReference frameTuple = cursor.getTuple();
+
+                rTreeBulkloader.add(frameTuple);
+            }
+        } finally {
+            cursor.close();
+        }
+
+        rTreeBulkloader.end();
+        return component;
+    }
+
+    @Override
+    public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+            throws HyracksDataException, IndexException {
+        List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
+        LSMRTreeOpContext rctx = createOpContext(NoOpOperationCallback.INSTANCE);
+        rctx.getComponentHolder().addAll(mergingComponents);
+        ITreeIndexCursor cursor = new LSMRTreeWithAntiMatterTuplesSearchCursor(ctx);
+        ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null);
+        search(rctx, cursor, (SearchPredicate) rtreeSearchPred);
+        rctx.setOperation(IndexOperation.MERGE);
+        LSMComponentFileReferences relMergeFileRefs = getMergeTargetFileName(mergingComponents);
+        ILSMIndexAccessorInternal accessor = new LSMRTreeWithAntiMatterTuplesAccessor(lsmHarness, rctx);
+        ioScheduler.scheduleOperation(new LSMRTreeMergeOperation(accessor, mergingComponents, cursor, relMergeFileRefs
+                .getInsertIndexFileReference(), null, null, callback));
+    }
+
+    @Override
+    public ILSMComponent merge(List<ILSMComponent> mergedComponents, ILSMIOOperation operation)
+            throws HyracksDataException, IndexException {
+        LSMRTreeMergeOperation mergeOp = (LSMRTreeMergeOperation) operation;
+        ITreeIndexCursor cursor = mergeOp.getCursor();
+        mergedComponents.addAll(mergeOp.getMergingComponents());
+
+        // Nothing to merge.
+        if (mergedComponents.size() <= 1) {
+            cursor.close();
+            return null;
+        }
+
+        // Bulk load the tuples from all on-disk RTrees into the new RTree.
+        LSMRTreeImmutableComponent component = createDiskComponent(componentFactory, mergeOp.getRTreeMergeTarget(),
+                null, null, true);
+        RTree mergedRTree = component.getRTree();
+        IIndexBulkLoader bulkloader = mergedRTree.createBulkLoader(1.0f, false, 0L);
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+                ITupleReference frameTuple = cursor.getTuple();
+                bulkloader.add(frameTuple);
+            }
+        } finally {
+            cursor.close();
+        }
+        bulkloader.end();
+        return component;
+    }
+
+    @Override
+    public ILSMIndexAccessorInternal createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new LSMRTreeWithAntiMatterTuplesAccessor(lsmHarness, createOpContext(modificationCallback));
+    }
+
+    public class LSMRTreeWithAntiMatterTuplesAccessor extends LSMTreeIndexAccessor {
+        public LSMRTreeWithAntiMatterTuplesAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx) {
+            super(lsmHarness, ctx);
+        }
+
+        @Override
+        public ITreeIndexCursor createSearchCursor() {
+            return new LSMRTreeWithAntiMatterTuplesSearchCursor(ctx);
+        }
+
+        public MultiComparator getMultiComparator() {
+            LSMRTreeOpContext concreteCtx = (LSMRTreeOpContext) ctx;
+            return concreteCtx.rtreeOpContext.cmp;
+        }
+    }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
+            throws TreeIndexException {
+        return new LSMRTreeWithAntiMatterTuplesBulkLoader(fillLevel, verifyInput, numElementsHint);
+    }
+
+    private ILSMComponent createBulkLoadTarget() throws HyracksDataException, IndexException {
+        LSMComponentFileReferences relFlushFileRefs = fileManager.getRelFlushFileReference();
+        return createDiskComponent(bulkLoaComponentFactory, relFlushFileRefs.getInsertIndexFileReference(), null, null,
+                true);
+    }
+
+    public class LSMRTreeWithAntiMatterTuplesBulkLoader implements IIndexBulkLoader {
+        private final ILSMComponent component;
+        private final IIndexBulkLoader bulkLoader;
+
+        public LSMRTreeWithAntiMatterTuplesBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+                throws TreeIndexException {
+            // 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);
+            }
+            bulkLoader = ((LSMRTreeImmutableComponent) component).getRTree().createBulkLoader(fillFactor, verifyInput,
+                    numElementsHint);
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws HyracksDataException, IndexException {
+            try {
+                bulkLoader.add(tuple);
+            } catch (IndexException e) {
+                handleException();
+                throw e;
+            } catch (HyracksDataException e) {
+                handleException();
+                throw e;
+            } catch (RuntimeException e) {
+                handleException();
+                throw e;
+            }
+        }
+
+        @Override
+        public void end() throws HyracksDataException, IndexException {
+            bulkLoader.end();
+            lsmHarness.addBulkLoadedComponent(component);
+        }
+
+        protected void handleException() throws HyracksDataException {
+            ((LSMRTreeImmutableComponent) component).getRTree().deactivate();
+            ((LSMRTreeImmutableComponent) component).getRTree().destroy();
+        }
+
+    }
+
+    @Override
+    public void markAsValid(ILSMComponent lsmComponent) throws HyracksDataException {
+        RTree rtree = ((LSMRTreeImmutableComponent) lsmComponent).getRTree();
+        forceFlushDirtyPages(rtree);
+        markAsValidInternal(rtree);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesComponentFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesComponentFactory.java
new file mode 100644
index 0000000..0149800
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesComponentFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public class LSMRTreeWithAntiMatterTuplesComponentFactory implements ILSMComponentFactory {
+    private final TreeIndexFactory<RTree> rtreeFactory;
+
+    public LSMRTreeWithAntiMatterTuplesComponentFactory(TreeIndexFactory<RTree> rtreeFactory) {
+        this.rtreeFactory = rtreeFactory;
+    }
+
+    @Override
+    public ILSMComponent createLSMComponentInstance(LSMComponentFileReferences cfr) throws IndexException {
+        return new LSMRTreeImmutableComponent(rtreeFactory.createIndexInstance(cfr.getInsertIndexFileReference()),
+                null, null);
+    }
+
+    @Override
+    public IBufferCache getBufferCache() {
+        return rtreeFactory.getBufferCache();
+    }
+}
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
new file mode 100644
index 0000000..10b982f
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
@@ -0,0 +1,126 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.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;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMRTreeWithAntiMatterTuplesFileManager extends AbstractLSMIndexFileManager {
+
+    private final TreeIndexFactory<? extends ITreeIndex> rtreeFactory;
+
+    public LSMRTreeWithAntiMatterTuplesFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider,
+            FileReference file, TreeIndexFactory<? extends ITreeIndex> rtreeFactory, int startIODeviceIndex) {
+        super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+        this.rtreeFactory = rtreeFactory;
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelFlushFileReference() {
+        Date date = new Date();
+        String ts = formatter.format(date);
+        // Begin timestamp and end timestamp are identical since it is a flush
+        return new LSMComponentFileReferences(createFlushFile(baseDir + ts + SPLIT_STRING + ts), null, null);
+    }
+
+    @Override
+    public LSMComponentFileReferences getRelMergeFileReference(String firstFileName, String lastFileName)
+            throws HyracksDataException {
+        String[] firstTimestampRange = firstFileName.split(SPLIT_STRING);
+        String[] lastTimestampRange = lastFileName.split(SPLIT_STRING);
+        // Get the range of timestamps by taking the earliest and the latest timestamps
+        return new LSMComponentFileReferences(createMergeFile(baseDir + firstTimestampRange[0] + SPLIT_STRING
+                + lastTimestampRange[1]), null, null);
+    }
+
+    private static FilenameFilter fileNameFilter = new FilenameFilter() {
+        public boolean accept(File dir, String name) {
+            return !name.startsWith(".");
+        }
+    };
+
+    @Override
+    public List<LSMComponentFileReferences> cleanupAndGetValidFiles() throws HyracksDataException, IndexException {
+        List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
+        ArrayList<ComparableFileName> allFiles = new ArrayList<ComparableFileName>();
+
+        // Gather files from all IODeviceHandles 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).
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            cleanupAndGetValidFilesInternal(dev, fileNameFilter, rtreeFactory, allFiles);
+        }
+
+        if (allFiles.isEmpty()) {
+            return validFiles;
+        }
+
+        if (allFiles.size() == 1) {
+            validFiles.add(new LSMComponentFileReferences(allFiles.get(0).fileRef, null, null));
+            return validFiles;
+        }
+
+        // Sorts files names from earliest to latest timestamp.
+        Collections.sort(allFiles);
+
+        List<ComparableFileName> validComparableFiles = new ArrayList<ComparableFileName>();
+        ComparableFileName last = allFiles.get(0);
+        validComparableFiles.add(last);
+        for (int i = 1; i < allFiles.size(); i++) {
+            ComparableFileName current = allFiles.get(i);
+            // The current start timestamp is greater than last stop timestamp so current is valid.
+            if (current.interval[0].compareTo(last.interval[1]) > 0) {
+                validComparableFiles.add(current);
+                last = current;
+            } else if (current.interval[0].compareTo(last.interval[0]) >= 0
+                    && current.interval[1].compareTo(last.interval[1]) <= 0) {
+                // The current file is completely contained in the interval of the 
+                // last file. Thus the last file must contain at least as much information 
+                // as the current file, so delete the current file.
+                current.fileRef.delete();
+            } else {
+                // This scenario should not be possible since timestamps are monotonically increasing.
+                throw new HyracksDataException("Found LSM files with overlapping timestamp intervals, "
+                        + "but the intervals were not contained by another file.");
+            }
+        }
+
+        // Sort valid files in reverse lexicographical order, such that newer files come first.
+        Collections.sort(validComparableFiles, recencyCmp);
+        for (ComparableFileName cmpFileName : validComparableFiles) {
+            validFiles.add(new LSMComponentFileReferences(cmpFileName.fileRef, null, null));
+        }
+
+        return validFiles;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
new file mode 100644
index 0000000..22e6929
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
@@ -0,0 +1,164 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+
+public class LSMRTreeWithAntiMatterTuplesFlushCursor implements ITreeIndexCursor {
+    private final TreeTupleSorter rTreeTupleSorter;
+    private final TreeTupleSorter bTreeTupleSorter;
+    private final int[] comparatorFields;
+    private final MultiComparator cmp;
+    private ITupleReference frameTuple;
+    private ITupleReference leftOverTuple;
+    private ITupleReference rtreeTuple;
+    private ITupleReference btreeTuple;
+    private boolean foundNext = false;
+
+    public LSMRTreeWithAntiMatterTuplesFlushCursor(TreeTupleSorter rTreeTupleSorter, TreeTupleSorter bTreeTupleSorter,
+            int[] comparatorFields, IBinaryComparatorFactory[] comparatorFactories) {
+        this.rTreeTupleSorter = rTreeTupleSorter;
+        this.bTreeTupleSorter = bTreeTupleSorter;
+        this.comparatorFields = comparatorFields;
+        cmp = MultiComparator.create(comparatorFactories);
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException {
+        if (foundNext) {
+            return true;
+        }
+        while (true) {
+            if (leftOverTuple != null && leftOverTuple == rtreeTuple) {
+                if (bTreeTupleSorter.hasNext()) {
+                    bTreeTupleSorter.next();
+                    btreeTuple = bTreeTupleSorter.getTuple();
+                } else {
+                    frameTuple = rtreeTuple;
+                    foundNext = true;
+                    leftOverTuple = null;
+                    return true;
+                }
+            } else if (leftOverTuple != null && leftOverTuple == btreeTuple) {
+                if (rTreeTupleSorter.hasNext()) {
+                    rTreeTupleSorter.next();
+                    rtreeTuple = rTreeTupleSorter.getTuple();
+                } else {
+                    frameTuple = btreeTuple;
+                    foundNext = true;
+                    leftOverTuple = null;
+                    return true;
+                }
+            } else {
+                if (rTreeTupleSorter.hasNext() && bTreeTupleSorter.hasNext()) {
+                    rTreeTupleSorter.next();
+                    bTreeTupleSorter.next();
+                    rtreeTuple = rTreeTupleSorter.getTuple();
+                    btreeTuple = bTreeTupleSorter.getTuple();
+                } else if (rTreeTupleSorter.hasNext()) {
+                    rTreeTupleSorter.next();
+                    rtreeTuple = rTreeTupleSorter.getTuple();
+                    frameTuple = rtreeTuple;
+                    leftOverTuple = null;
+                    foundNext = true;
+                    return true;
+                } else if (bTreeTupleSorter.hasNext()) {
+                    bTreeTupleSorter.next();
+                    btreeTuple = bTreeTupleSorter.getTuple();
+                    frameTuple = btreeTuple;
+                    leftOverTuple = null;
+                    foundNext = true;
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            int c = cmp.selectiveFieldCompare(rtreeTuple, btreeTuple, comparatorFields);
+            if (c == 0) {
+                leftOverTuple = null;
+                continue;
+            } else if (c < 0) {
+                frameTuple = rtreeTuple;
+                leftOverTuple = btreeTuple;
+                foundNext = true;
+                return true;
+            } else {
+                frameTuple = btreeTuple;
+                leftOverTuple = rtreeTuple;
+                foundNext = true;
+                return true;
+            }
+        }
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        foundNext = false;
+
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return frameTuple;
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        return null;
+    }
+
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public boolean exclusiveLatchNodes() {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
new file mode 100644
index 0000000..47d00c0
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
@@ -0,0 +1,249 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
+
+public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCursor {
+
+    private RTreeSearchCursor memRTreeCursor;
+    private BTreeRangeSearchCursor memBTreeCursor;
+    private RangePredicate btreeRangePredicate;
+    private ITreeIndexAccessor memBTreeAccessor;
+    private boolean foundNext;
+    private ITupleReference frameTuple;
+    private int[] comparatorFields;
+    private MultiComparator btreeCmp;
+
+    public LSMRTreeWithAntiMatterTuplesSearchCursor(ILSMIndexOperationContext opCtx) {
+        super(opCtx);
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        LSMRTreeCursorInitialState lsmInitialState = (LSMRTreeCursorInitialState) initialState;
+        cmp = lsmInitialState.getHilbertCmp();
+        btreeCmp = lsmInitialState.getBTreeCmp();
+        int numDiskRTrees = lsmInitialState.getNumberOfTrees();
+        rangeCursors = new RTreeSearchCursor[numDiskRTrees];
+        for (int i = 0; i < numDiskRTrees; i++) {
+            rangeCursors[i] = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState
+                    .getRTreeInteriorFrameFactory().createFrame(), (IRTreeLeafFrame) lsmInitialState
+                    .getRTreeLeafFrameFactory().createFrame());
+        }
+        includeMemComponent = lsmInitialState.getIncludeMemComponent();
+        operationalComponents = lsmInitialState.getOperationalComponents();
+        if (includeMemComponent) {
+            memRTreeCursor = new RTreeSearchCursor((IRTreeInteriorFrame) lsmInitialState.getRTreeInteriorFrameFactory()
+                    .createFrame(), (IRTreeLeafFrame) lsmInitialState.getRTreeLeafFrameFactory().createFrame());
+            memBTreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) lsmInitialState.getBTreeLeafFrameFactory()
+                    .createFrame(), false);
+            memBTreeAccessor = lsmInitialState.getBTreeAccessors()[0];
+            btreeRangePredicate = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);
+        }
+        lsmHarness = lsmInitialState.getLSMHarness();
+        comparatorFields = lsmInitialState.getComparatorFields();
+        setPriorityQueueComparator();
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException, IndexException {
+        if (includeMemComponent) {
+            if (foundNext) {
+                return true;
+            }
+            while (memRTreeCursor.hasNext()) {
+                memRTreeCursor.next();
+                ITupleReference memRTreeTuple = memRTreeCursor.getTuple();
+                if (searchMemBTree(memRTreeTuple)) {
+                    foundNext = true;
+                    frameTuple = memRTreeTuple;
+                    return true;
+                }
+            }
+            while (super.hasNext()) {
+                super.next();
+                ITupleReference diskRTreeTuple = super.getTuple();
+                if (searchMemBTree(diskRTreeTuple)) {
+                    foundNext = true;
+                    frameTuple = diskRTreeTuple;
+                    return true;
+                }
+            }
+        } else {
+            return super.hasNext();
+        }
+
+        return false;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        if (includeMemComponent) {
+            foundNext = false;
+        } else {
+            super.next();
+        }
+
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        if (includeMemComponent) {
+            return frameTuple;
+        } else {
+            return super.getTuple();
+        }
+
+    }
+
+    @Override
+    public void reset() throws HyracksDataException, IndexException {
+        if (includeMemComponent) {
+            memRTreeCursor.reset();
+            memBTreeCursor.reset();
+        }
+        super.reset();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (includeMemComponent) {
+            memRTreeCursor.close();
+            memBTreeCursor.close();
+        }
+        super.close();
+    }
+
+    public ITreeIndexCursor getMemRTreeCursor() {
+        return memRTreeCursor;
+    }
+
+    @Override
+    protected int compare(MultiComparator cmp, ITupleReference tupleA, ITupleReference tupleB) {
+        return cmp.selectiveFieldCompare(tupleA, tupleB, comparatorFields);
+    }
+
+    private boolean searchMemBTree(ITupleReference tuple) throws HyracksDataException {
+        try {
+            btreeRangePredicate.setHighKey(tuple, true);
+            btreeRangePredicate.setLowKey(tuple, true);
+            memBTreeAccessor.search(memBTreeCursor, btreeRangePredicate);
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+        try {
+            if (memBTreeCursor.hasNext()) {
+                return false;
+            } else {
+                return true;
+            }
+        } finally {
+            memBTreeCursor.close();
+        }
+    }
+
+    @Override
+    protected void setPriorityQueueComparator() {
+        if (pqCmp == null || cmp != pqCmp.getMultiComparator()) {
+            pqCmp = new PriorityQueueHilbertComparator(cmp, comparatorFields);
+        }
+    }
+
+    public class PriorityQueueHilbertComparator extends PriorityQueueComparator {
+
+        private final int[] comparatorFields;
+
+        public PriorityQueueHilbertComparator(MultiComparator cmp, int[] comparatorFields) {
+            super(cmp);
+            this.comparatorFields = comparatorFields;
+        }
+
+        @Override
+        public int compare(PriorityQueueElement elementA, PriorityQueueElement elementB) {
+            int result = cmp.selectiveFieldCompare(elementA.getTuple(), elementB.getTuple(), comparatorFields);
+            if (result != 0) {
+                return result;
+            }
+            if (elementA.getCursorIndex() > elementB.getCursorIndex()) {
+                return 1;
+            } else {
+                return -1;
+            }
+        }
+    }
+
+    @Override
+    protected void checkPriorityQueue() throws HyracksDataException, IndexException {
+        while (!outputPriorityQueue.isEmpty() || needPush == true) {
+            if (!outputPriorityQueue.isEmpty()) {
+                PriorityQueueElement checkElement = outputPriorityQueue.peek();
+                // If there is no previous tuple or the previous tuple can be ignored
+                if (outputElement == null) {
+                    if (isDeleted(checkElement)) {
+                        // If the key has been deleted then pop it and set needPush to true.
+                        // We cannot push immediately because the tuple may be
+                        // modified if hasNext() is called
+                        outputElement = outputPriorityQueue.poll();
+                        needPush = true;
+                    } else {
+                        break;
+                    }
+                } else {
+                    // Compare the previous tuple and the head tuple in the PQ
+                    if (compare(cmp, outputElement.getTuple(), checkElement.getTuple()) == 0) {
+                        // If the previous tuple and the head tuple are
+                        // identical
+                        // then pop the head tuple and push the next tuple from
+                        // the tree of head tuple
+
+                        // the head element of PQ is useless now
+                        PriorityQueueElement e = outputPriorityQueue.poll();
+                        pushIntoPriorityQueue(e);
+                    } else {
+                        // If the previous tuple and the head tuple are different
+                        // the info of previous tuple is useless
+                        if (needPush == true) {
+                            pushIntoPriorityQueue(outputElement);
+                            needPush = false;
+                        }
+                        outputElement = null;
+                    }
+                }
+            } else {
+                // the priority queue is empty and needPush
+                pushIntoPriorityQueue(outputElement);
+                needPush = false;
+                outputElement = null;
+            }
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/RTreeFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/RTreeFactory.java
new file mode 100644
index 0000000..71e228b
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/RTreeFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManagerFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class RTreeFactory extends TreeIndexFactory<RTree> {
+
+    public RTreeFactory(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            IFreePageManagerFactory freePageManagerFactory, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, IBinaryComparatorFactory[] cmpFactories, int fieldCount) {
+        super(bufferCache, fileMapProvider, freePageManagerFactory, interiorFrameFactory, leafFrameFactory,
+                cmpFactories, fieldCount);
+    }
+
+    @Override
+    public RTree createIndexInstance(FileReference file) throws IndexException {
+        return new RTree(bufferCache, fileMapProvider, freePageManagerFactory.createFreePageManager(),
+                interiorFrameFactory, leafFrameFactory, cmpFactories, fieldCount, file);
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
new file mode 100644
index 0000000..294c2b8
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
@@ -0,0 +1,225 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+
+public class TreeTupleSorter implements ITreeIndexCursor {
+    private final static int INITIAL_SIZE = 1000000;
+    private int numTuples;
+    private int currentTupleIndex;
+    private int[] tPointers;
+    private IBufferCache bufferCache;
+    private final ITreeIndexFrame leafFrame1;
+    private final ITreeIndexFrame leafFrame2;
+    private ITreeIndexTupleReference frameTuple1;
+    private ITreeIndexTupleReference frameTuple2;
+    private final int fileId;
+    private final static int ARRAY_GROWTH = 1000000; // Must be at least of size 2
+    private final int[] comparatorFields;
+    private final MultiComparator cmp;
+
+    public TreeTupleSorter(int fileId, IBinaryComparatorFactory[] comparatorFactories, ITreeIndexFrame leafFrame1,
+            ITreeIndexFrame leafFrame2, IBufferCache bufferCache, int[] comparatorFields) {
+        this.fileId = fileId;
+        this.leafFrame1 = leafFrame1;
+        this.leafFrame2 = leafFrame2;
+        this.bufferCache = bufferCache;
+        this.comparatorFields = comparatorFields;
+        tPointers = new int[INITIAL_SIZE * 2];
+        frameTuple1 = leafFrame1.createTupleReference();
+        frameTuple2 = leafFrame2.createTupleReference();
+        currentTupleIndex = 0;
+        cmp = MultiComparator.create(comparatorFactories);
+    }
+
+    public void reset() {
+        numTuples = 0;
+        currentTupleIndex = 0;
+    }
+
+    public boolean hasNext() throws HyracksDataException {
+        if (numTuples <= currentTupleIndex) {
+            return false;
+        }
+        // We don't latch pages since this code is only used by flush () before
+        // bulk-loading the r-tree to disk and flush is not concurrent.
+        //
+        ICachedPage node1 = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, tPointers[currentTupleIndex * 2]),
+                false);
+        try {
+            leafFrame1.setPage(node1);
+            frameTuple1.resetByTupleOffset(leafFrame1.getBuffer(), tPointers[currentTupleIndex * 2 + 1]);
+        } finally {
+            bufferCache.unpin(node1);
+        }
+        return true;
+    }
+
+    public void next() {
+        currentTupleIndex++;
+    }
+
+    public ITupleReference getTuple() {
+        return frameTuple1;
+    }
+
+    public void insertTupleEntry(int pageId, int tupleOffset) {
+        if (numTuples * 2 == tPointers.length) {
+            int[] newData = new int[tPointers.length + ARRAY_GROWTH];
+            System.arraycopy(tPointers, 0, newData, 0, tPointers.length);
+            tPointers = newData;
+        }
+
+        tPointers[numTuples * 2] = pageId;
+        tPointers[numTuples * 2 + 1] = tupleOffset;
+        numTuples++;
+    }
+
+    public void sort() throws HyracksDataException {
+        sort(tPointers, 0, numTuples);
+    }
+
+    private void sort(int[] tPointers, int offset, int length) throws HyracksDataException {
+        int m = offset + (length >> 1);
+        int mi = tPointers[m * 2];
+        int mj = tPointers[m * 2 + 1];
+
+        int a = offset;
+        int b = a;
+        int c = offset + length - 1;
+        int d = c;
+        while (true) {
+            while (b <= c) {
+                int cmp = compare(tPointers, b, mi, mj);
+                if (cmp > 0) {
+                    break;
+                }
+                if (cmp == 0) {
+                    swap(tPointers, a++, b);
+                }
+                ++b;
+            }
+            while (c >= b) {
+                int cmp = compare(tPointers, c, mi, mj);
+                if (cmp < 0) {
+                    break;
+                }
+                if (cmp == 0) {
+                    swap(tPointers, c, d--);
+                }
+                --c;
+            }
+            if (b > c)
+                break;
+            swap(tPointers, b++, c--);
+        }
+
+        int s;
+        int n = offset + length;
+        s = Math.min(a - offset, b - a);
+        vecswap(tPointers, offset, b - s, s);
+        s = Math.min(d - c, n - d - 1);
+        vecswap(tPointers, b, n - s, s);
+
+        if ((s = b - a) > 1) {
+            sort(tPointers, offset, s);
+        }
+        if ((s = d - c) > 1) {
+            sort(tPointers, n - s, s);
+        }
+    }
+
+    private void swap(int x[], int a, int b) {
+        for (int i = 0; i < 2; ++i) {
+            int t = x[a * 2 + i];
+            x[a * 2 + i] = x[b * 2 + i];
+            x[b * 2 + i] = t;
+        }
+    }
+
+    private void vecswap(int x[], int a, int b, int n) {
+        for (int i = 0; i < n; i++, a++, b++) {
+            swap(x, a, b);
+        }
+    }
+
+    private int compare(int[] tPointers, int tp1, int tp2i, int tp2j) throws HyracksDataException {
+        int i1 = tPointers[tp1 * 2];
+        int j1 = tPointers[tp1 * 2 + 1];
+
+        int i2 = tp2i;
+        int j2 = tp2j;
+
+        ICachedPage node1 = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i1), false);
+        leafFrame1.setPage(node1);
+        ICachedPage node2 = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i2), false);
+        leafFrame2.setPage(node2);
+
+        try {
+            frameTuple1.resetByTupleOffset(leafFrame1.getBuffer(), j1);
+            frameTuple2.resetByTupleOffset(leafFrame2.getBuffer(), j2);
+
+            return cmp.selectiveFieldCompare(frameTuple1, frameTuple2, comparatorFields);
+
+        } finally {
+            bufferCache.unpin(node1);
+            bufferCache.unpin(node2);
+        }
+    }
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        // do nothing
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        // do nothing
+    }
+
+    @Override
+    public ICachedPage getPage() {
+        return null;
+    }
+
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        // do nothing
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        // do nothing
+    }
+
+    @Override
+    public boolean exclusiveLatchNodes() {
+        return false;
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
new file mode 100644
index 0000000..1852b51
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriter.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 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.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class LSMRTreeCopyTupleWriter extends LSMRTreeTupleWriter {
+    public LSMRTreeCopyTupleWriter(ITypeTraits[] typeTraits) {
+        // Third parameter is never used locally, just give false.
+        super(typeTraits, false);
+    }
+
+    @Override
+    public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
+        int tupleSize = bytesRequired(tuple);
+        byte[] buf = tuple.getFieldData(0);
+        int tupleStartOff = ((LSMRTreeTupleReference) tuple).getTupleStart();
+        System.arraycopy(buf, tupleStartOff, targetBuf, targetOff, tupleSize);
+        return tupleSize;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
new file mode 100644
index 0000000..39a8e4d
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeCopyTupleWriterFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2010 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.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+
+public class LSMRTreeCopyTupleWriterFactory extends TypeAwareTupleWriterFactory {
+    private static final long serialVersionUID = 1L;
+    private final ITypeTraits[] typeTraits;
+
+    public LSMRTreeCopyTupleWriterFactory(ITypeTraits[] typeTraits) {
+        super(typeTraits);
+        this.typeTraits = typeTraits;
+    }
+
+    @Override
+    public ITreeIndexTupleWriter createTupleWriter() {
+        return new LSMRTreeCopyTupleWriter(typeTraits);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
new file mode 100644
index 0000000..70072e1
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleReference.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 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.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+
+public class LSMRTreeTupleReference extends TypeAwareTupleReference implements ILSMTreeTupleReference {
+
+    public LSMRTreeTupleReference(ITypeTraits[] typeTraits) {
+        super(typeTraits);
+    }
+
+    @Override
+    protected int getNullFlagsBytes() {
+        // +1.0 is for matter/antimatter bit.
+        return (int) Math.ceil((fieldCount + 1.0) / 8.0);
+    }
+
+    @Override
+    public boolean isAntimatter() {
+        // Check if the leftmost bit is 0 or 1.
+        final byte mask = (byte) (1 << 7);
+        if ((buf.array()[tupleStartOff] & mask) != 0) {
+            return true;
+        }
+        return false;
+    }
+
+    public int getTupleStart() {
+        return tupleStartOff;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
new file mode 100644
index 0000000..932a307
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriter.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2010 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.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
+
+public class LSMRTreeTupleWriter extends RTreeTypeAwareTupleWriter {
+    private final boolean isAntimatter;
+
+    public LSMRTreeTupleWriter(ITypeTraits[] typeTraits, boolean isAntimatter) {
+        super(typeTraits);
+        this.isAntimatter = isAntimatter;
+    }
+
+    @Override
+    public ITreeIndexTupleReference createTupleReference() {
+        return new LSMRTreeTupleReference(typeTraits);
+    }
+
+    @Override
+    public int bytesRequired(ITupleReference tuple) {
+        return super.bytesRequired(tuple);
+    }
+
+    @Override
+    public int writeTuple(ITupleReference tuple, byte[] targetBuf, int targetOff) {
+        int bytesWritten = super.writeTuple(tuple, targetBuf, targetOff);
+        if (isAntimatter) {
+            setAntimatterBit(targetBuf, targetOff);
+        }
+        return bytesWritten;
+    }
+
+    @Override
+    protected int getNullFlagsBytes(int numFields) {
+        // +1.0 is for matter/antimatter bit.
+        return (int) Math.ceil(((double) numFields + 1.0) / 8.0);
+    }
+
+    @Override
+    protected int getNullFlagsBytes(ITupleReference tuple) {
+        // +1.0 is for matter/antimatter bit.
+        return (int) Math.ceil(((double) tuple.getFieldCount() + 1.0) / 8.0);
+    }
+
+    protected void setAntimatterBit(byte[] targetBuf, int targetOff) {
+        // Set leftmost bit to 1.
+        targetBuf[targetOff] = (byte) (targetBuf[targetOff] | (1 << 7));
+    }
+
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
new file mode 100644
index 0000000..493d368
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMRTreeTupleWriterFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2010 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.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+
+public class LSMRTreeTupleWriterFactory extends TypeAwareTupleWriterFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final ITypeTraits[] typeTraits;
+    private final boolean isDelete;
+
+    public LSMRTreeTupleWriterFactory(ITypeTraits[] typeTraits, boolean isDelete) {
+        super(typeTraits);
+        this.typeTraits = typeTraits;
+        this.isDelete = isDelete;
+    }
+
+    @Override
+    public ITreeIndexTupleWriter createTupleWriter() {
+        return new LSMRTreeTupleWriter(typeTraits, isDelete);
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMTypeAwareTupleWriterFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMTypeAwareTupleWriterFactory.java
new file mode 100644
index 0000000..876df56
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/tuples/LSMTypeAwareTupleWriterFactory.java
@@ -0,0 +1,30 @@
+package edu.uci.ics.hyracks.storage.am.lsm.rtree.tuples;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
+
+public class LSMTypeAwareTupleWriterFactory extends TypeAwareTupleWriterFactory {
+
+	private static final long serialVersionUID = 1L;
+	private ITypeTraits[] typeTraits;
+	private final boolean isDelete;
+	
+	public LSMTypeAwareTupleWriterFactory(ITypeTraits[] typeTraits, boolean isDelete) {
+		super(typeTraits);
+		this.typeTraits = typeTraits;
+		this.isDelete = isDelete;
+	}
+
+	@Override
+	public ITreeIndexTupleWriter createTupleWriter() {
+	    if (isDelete) {
+	        return new TypeAwareTupleWriter(typeTraits);
+	    } else {
+	        return new RTreeTypeAwareTupleWriter(typeTraits);
+	    }
+	}
+
+}
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
new file mode 100644
index 0000000..6c9fce6
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -0,0 +1,211 @@
+/*
+ * Copyright 2009-2012 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.utils;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.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;
+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.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTree;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuples;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuplesFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.RTreeFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.tuples.LSMRTreeCopyTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.tuples.LSMRTreeTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.tuples.LSMTypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.am.rtree.linearize.HilbertDoubleComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.linearize.ZCurveDoubleComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.linearize.ZCurveIntComparatorFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class LSMRTreeUtils {
+    public static LSMRTree createLSMTree(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory) throws TreeIndexException {
+        return createLSMTree(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache, diskFileMapProvider,
+                typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory, 0);
+    }
+
+    public static LSMRTree createLSMTree(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws TreeIndexException {
+        LSMTypeAwareTupleWriterFactory rtreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, false);
+        LSMTypeAwareTupleWriterFactory btreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, true);
+
+        ITreeIndexFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(rtreeTupleWriterFactory,
+                valueProviderFactories, rtreePolicyType);
+        ITreeIndexFrameFactory rtreeLeafFrameFactory = new RTreeNSMLeafFrameFactory(rtreeTupleWriterFactory,
+                valueProviderFactories, rtreePolicyType);
+
+        ITreeIndexFrameFactory btreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(btreeTupleWriterFactory);
+        ITreeIndexFrameFactory btreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(btreeTupleWriterFactory);
+
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        LinkedListFreePageManagerFactory freePageManagerFactory = new LinkedListFreePageManagerFactory(diskBufferCache,
+                metaFrameFactory);
+
+        TreeIndexFactory<RTree> diskRTreeFactory = new RTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, rtreeCmpFactories,
+                typeTraits.length);
+        TreeIndexFactory<BTree> diskBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeCmpFactories,
+                typeTraits.length);
+
+        int[] comparatorFields = { 0 };
+        IBinaryComparatorFactory[] linearizerArray = { linearizeCmpFactory };
+
+        int[] bloomFilterKeyFields = new int[btreeCmpFactories.length];
+        for (int i = 0; i < btreeCmpFactories.length; i++) {
+            bloomFilterKeyFields[i] = i;
+        }
+        BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, diskFileMapProvider,
+                bloomFilterKeyFields);
+
+        ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(ioManager, diskFileMapProvider, file,
+                diskRTreeFactory, diskBTreeFactory, startIODeviceIndex);
+        LSMRTree lsmTree = new LSMRTree(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory,
+                rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager,
+                diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, diskFileMapProvider, typeTraits.length,
+                rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
+                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        return lsmTree;
+    }
+
+    public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizerCmpFactory) throws TreeIndexException {
+        return createLSMTreeWithAntiMatterTuples(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache,
+                diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
+                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+                linearizerCmpFactory, 0);
+    }
+
+    public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILinearizeComparatorFactory linearizerCmpFactory, int startIODeviceIndex) throws TreeIndexException {
+
+        LSMRTreeTupleWriterFactory rtreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
+        LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
+
+        LSMRTreeCopyTupleWriterFactory copyTupleWriterFactory = new LSMRTreeCopyTupleWriterFactory(typeTraits);
+
+        ITreeIndexFrameFactory rtreeInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(rtreeTupleWriterFactory,
+                valueProviderFactories, rtreePolicyType);
+        ITreeIndexFrameFactory rtreeLeafFrameFactory = new RTreeNSMLeafFrameFactory(rtreeTupleWriterFactory,
+                valueProviderFactories, rtreePolicyType);
+
+        ITreeIndexFrameFactory btreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(btreeTupleWriterFactory);
+        ITreeIndexFrameFactory btreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(btreeTupleWriterFactory);
+
+        ITreeIndexFrameFactory copyTupleLeafFrameFactory = new RTreeNSMLeafFrameFactory(copyTupleWriterFactory,
+                valueProviderFactories, rtreePolicyType);
+
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        LinkedListFreePageManagerFactory freePageManagerFactory = new LinkedListFreePageManagerFactory(diskBufferCache,
+                metaFrameFactory);
+
+        TreeIndexFactory<RTree> diskRTreeFactory = new RTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, rtreeInteriorFrameFactory, copyTupleLeafFrameFactory, rtreeCmpFactories,
+                typeTraits.length);
+
+        TreeIndexFactory<RTree> bulkLoadRTreeFactory = new RTreeFactory(diskBufferCache, diskFileMapProvider,
+                freePageManagerFactory, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, rtreeCmpFactories,
+                typeTraits.length);
+
+        // The first field is for the sorted curve (e.g. Hilbert curve), and the
+        // second field is for the primary key.
+        int[] comparatorFields = { 0, btreeCmpFactories.length - 1 };
+        IBinaryComparatorFactory[] linearizerArray = { linearizerCmpFactory,
+                btreeCmpFactories[btreeCmpFactories.length - 1] };
+
+        ILSMIndexFileManager fileNameManager = new LSMRTreeWithAntiMatterTuplesFileManager(ioManager,
+                diskFileMapProvider, file, diskRTreeFactory, startIODeviceIndex);
+        LSMRTreeWithAntiMatterTuples lsmTree = new LSMRTreeWithAntiMatterTuples(memBufferCache, memFreePageManager,
+                rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
+                fileNameManager, diskRTreeFactory, bulkLoadRTreeFactory, diskFileMapProvider, typeTraits.length,
+                rtreeCmpFactories, btreeCmpFactories, linearizerCmpFactory, comparatorFields, linearizerArray,
+                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        return lsmTree;
+    }
+
+    public static ILinearizeComparatorFactory proposeBestLinearizer(ITypeTraits[] typeTraits, int numKeyFields)
+            throws TreeIndexException {
+        for (int i = 0; i < numKeyFields; i++) {
+            if (!(typeTraits[i].getClass().equals(typeTraits[0].getClass()))) {
+                throw new TreeIndexException("Cannot propose linearizer if dimensions have different types");
+            }
+        }
+
+        if (numKeyFields / 2 == 2 && (typeTraits[0].getClass() == DoublePointable.TYPE_TRAITS.getClass())) {
+            return new HilbertDoubleComparatorFactory(2);
+        } else if (typeTraits[0].getClass() == DoublePointable.TYPE_TRAITS.getClass()) {
+            return new ZCurveDoubleComparatorFactory(numKeyFields / 2);
+        } else if (typeTraits[0].getClass() == IntegerPointable.TYPE_TRAITS.getClass()) {
+            return new ZCurveIntComparatorFactory(numKeyFields / 2);
+        }
+
+        throw new TreeIndexException("Cannot propose linearizer");
+    }
+}
diff --git a/hyracks/hyracks-storage-am-rtree/pom.xml b/hyracks/hyracks-storage-am-rtree/pom.xml
index 6c2d734..9ac1dd4 100644
--- a/hyracks/hyracks-storage-am-rtree/pom.xml
+++ b/hyracks/hyracks-storage-am-rtree/pom.xml
@@ -2,7 +2,6 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-storage-am-rtree</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-storage-am-rtree</name>
 
   <parent>
@@ -32,27 +31,27 @@
   		<version>0.2.3-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
-  	</dependency>  	
+  	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-common</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
-  	</dependency>  	
+  	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-dataflow-std</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
-  	</dependency>  	
+  	</dependency>
   	<dependency>
   		<groupId>junit</groupId>
   		<artifactId>junit</artifactId>
   		<version>4.8.1</version>
   		<type>jar</type>
   		<scope>test</scope>
-  	</dependency>  	  		
+  	</dependency>
   </dependencies>
 </project>
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreeInteriorFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreeInteriorFrame.java
index 5f333f3..59c047c 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreeInteriorFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreeInteriorFrame.java
@@ -22,9 +22,9 @@
 
 public interface IRTreeInteriorFrame extends IRTreeFrame {
 
-    public boolean findBestChild(ITupleReference tuple, MultiComparator cmp);
+    public int findBestChild(ITupleReference tuple, MultiComparator cmp);
 
-    public int getBestChildPageId();
+    public boolean checkIfEnlarementIsNeeded(ITupleReference tuple, MultiComparator cmp);
 
     public int getChildPageId(int tupleIndex);
 
@@ -36,9 +36,6 @@
 
     public void adjustKey(ITupleReference tuple, int tupleIndex, MultiComparator cmp) throws TreeIndexException;
 
-    public boolean recomputeMBR(ITupleReference tuple, int tupleIndex, MultiComparator cmp);
-
     public void enlarge(ITupleReference tuple, MultiComparator cmp);
 
-    boolean checkEnlargement(ITupleReference tuple, MultiComparator cmp);
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreeLeafFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreeLeafFrame.java
index 3005785..858a40d 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreeLeafFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreeLeafFrame.java
@@ -20,8 +20,9 @@
 
 public interface IRTreeLeafFrame extends IRTreeFrame {
 
-	public int findTupleIndex(ITupleReference tuple, MultiComparator cmp);
+    public int findTupleIndex(ITupleReference tuple, MultiComparator cmp);
 
-	public boolean intersect(ITupleReference tuple, int tupleIndex,
-			MultiComparator cmp);
+    public boolean intersect(ITupleReference tuple, int tupleIndex, MultiComparator cmp);
+
+    public ITupleReference getBeforeTuple(ITupleReference tuple, int targetTupleIndex, MultiComparator cmp);
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreePolicy.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreePolicy.java
new file mode 100644
index 0000000..a0cc5e8
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/api/IRTreePolicy.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 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.rtree.api;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ISlotManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+
+public interface IRTreePolicy {
+    public void split(ITreeIndexFrame leftFrame, ByteBuffer buf, ITreeIndexFrame rightFrame, ISlotManager slotManager,
+            ITreeIndexTupleReference frameTuple, ITupleReference tuple, ISplitKey splitKey);
+
+    public int findBestChildPosition(ITreeIndexFrame frame, ITupleReference tuple, ITreeIndexTupleReference frameTuple,
+            MultiComparator cmp);
+}
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelper.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelper.java
index 0470da9..5c3b314 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelper.java
@@ -19,24 +19,29 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 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;
 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.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
 public class RTreeDataflowHelper extends TreeIndexDataflowHelper {
 
     private final IPrimitiveValueProviderFactory[] valueProviderFactories;
+    private final RTreePolicyType rtreePolicyType;
 
     public RTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            IPrimitiveValueProviderFactory[] valueProviderFactories) {
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType) {
         super(opDesc, ctx, partition);
         this.valueProviderFactories = valueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
     }
 
     @Override
     public ITreeIndex createIndexInstance() throws HyracksDataException {
-        return RTreeUtils.createRTree(treeOpDesc.getStorageManager().getBufferCache(ctx),
-                treeOpDesc.getTreeIndexTypeTraits(), valueProviderFactories,
-                treeOpDesc.getTreeIndexComparatorFactories());
+        AbstractTreeIndexOperatorDescriptor treeOpDesc = (AbstractTreeIndexOperatorDescriptor) opDesc;
+        return RTreeUtils.createRTree(treeOpDesc.getStorageManager().getBufferCache(ctx), treeOpDesc
+                .getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
+                valueProviderFactories, treeOpDesc.getTreeIndexComparatorFactories(), rtreePolicyType, file);
     }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelperFactory.java
index 6b9fd4c..06af8ee 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelperFactory.java
@@ -20,20 +20,24 @@
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 public class RTreeDataflowHelperFactory implements IIndexDataflowHelperFactory {
 
     private static final long serialVersionUID = 1L;
 
     private final IPrimitiveValueProviderFactory[] valueProviderFactories;
+    private final RTreePolicyType rtreePolicyType;
 
-    public RTreeDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories) {
+    public RTreeDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType) {
         this.valueProviderFactories = valueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
     }
 
     @Override
     public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition) {
-        return new RTreeDataflowHelper(opDesc, ctx, partition, valueProviderFactories);
+        return new RTreeDataflowHelper(opDesc, ctx, partition, valueProviderFactories, rtreePolicyType);
     }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index d9b7b97..d718c69 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -23,12 +23,13 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.NoOpLocalResourceFactoryProvider;
 
 public class RTreeSearchOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
@@ -37,20 +38,22 @@
     protected int[] keyFields; // fields in input tuple to be used as keys
 
     public RTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
             IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
-            IOperationCallbackProvider opCallbackProvider) {
-        super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, null, retainInput, opCallbackProvider);
+            ISearchOperationCallbackFactory searchOpCallbackFactory) {
+        super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, dataflowHelperFactory, null, retainInput,
+                NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackFactory,
+                NoOpOperationCallbackFactory.INSTANCE);
+
         this.keyFields = keyFields;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new RTreeSearchOperatorNodePushable(this, ctx, opCallbackProvider, partition, recordDescProvider,
-                keyFields);
+        return new RTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, keyFields);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 3781037..de4961b 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -17,22 +17,21 @@
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexSearchOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
-public class RTreeSearchOperatorNodePushable extends TreeIndexSearchOperatorNodePushable {
+public class RTreeSearchOperatorNodePushable extends IndexSearchOperatorNodePushable {
     protected PermutingFrameTupleReference searchKey;
     protected MultiComparator cmp;
 
     public RTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
-            IOperationCallbackProvider opCallbackProvider, int partition, IRecordDescriptorProvider recordDescProvider,
-            int[] keyFields) {
+            int partition, IRecordDescriptorProvider recordDescProvider, int[] keyFields) {
         super(opDesc, ctx, partition, recordDescProvider);
         if (keyFields != null && keyFields.length > 0) {
             searchKey = new PermutingFrameTupleReference();
@@ -42,6 +41,7 @@
 
     @Override
     protected ISearchPredicate createSearchPredicate() {
+        ITreeIndex treeIndex = (ITreeIndex) index;
         cmp = RTreeUtils.getSearchMultiComparator(treeIndex.getComparatorFactories(), searchKey);
         return new SearchPredicate(searchKey, cmp);
     }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RStarTreePolicy.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RStarTreePolicy.java
new file mode 100644
index 0000000..aafecd5
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RStarTreePolicy.java
@@ -0,0 +1,354 @@
+/*
+ * Copyright 2009-2010 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.rtree.frames;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISlotManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreePolicy;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.EntriesOrder;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSplitKey;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.Rectangle;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.TupleEntryArrayList;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.UnorderedSlotManager;
+import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
+
+public class RStarTreePolicy implements IRTreePolicy {
+
+    private TupleEntryArrayList tupleEntries1;
+    private TupleEntryArrayList tupleEntries2;
+    private Rectangle[] rec;
+
+    private static final int nearMinimumOverlapFactor = 32;
+    private static final double splitFactor = 0.4;
+    private static final int numTuplesEntries = 100;
+
+    private final ITreeIndexTupleWriter tupleWriter;
+    private final IPrimitiveValueProvider[] keyValueProviders;
+    private ITreeIndexTupleReference cmpFrameTuple;
+    private final int totalFreeSpaceOff;
+
+    public RStarTreePolicy(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders,
+            ITreeIndexTupleReference cmpFrameTuple, int totalFreeSpaceOff) {
+        this.tupleWriter = tupleWriter;
+        this.keyValueProviders = keyValueProviders;
+        this.totalFreeSpaceOff = totalFreeSpaceOff;
+        this.cmpFrameTuple = cmpFrameTuple;
+        tupleEntries1 = new TupleEntryArrayList(numTuplesEntries, numTuplesEntries);
+        tupleEntries2 = new TupleEntryArrayList(numTuplesEntries, numTuplesEntries);
+        rec = new Rectangle[4];
+        for (int i = 0; i < 4; i++) {
+            rec[i] = new Rectangle(keyValueProviders.length / 2);
+        }
+    }
+
+    @Override
+    public void split(ITreeIndexFrame leftFrame, ByteBuffer buf, ITreeIndexFrame rightFrame, ISlotManager slotManager,
+            ITreeIndexTupleReference frameTuple, ITupleReference tuple, ISplitKey splitKey) {
+        RTreeSplitKey rTreeSplitKey = ((RTreeSplitKey) splitKey);
+        RTreeTypeAwareTupleWriter rTreeTupleWriterleftRTreeFrame = ((RTreeTypeAwareTupleWriter) tupleWriter);
+        RTreeTypeAwareTupleWriter rTreeTupleWriterRightFrame = ((RTreeTypeAwareTupleWriter) rightFrame.getTupleWriter());
+
+        RTreeNSMFrame leftRTreeFrame = ((RTreeNSMFrame) leftFrame);
+
+        // calculations are based on the R*-tree paper
+        int m = (int) Math.floor((leftRTreeFrame.getTupleCount() + 1) * splitFactor);
+        int splitDistribution = leftRTreeFrame.getTupleCount() - (2 * m) + 2;
+
+        // to calculate the minimum margin in order to pick the split axis
+        double minMargin = Double.MAX_VALUE;
+        int splitAxis = 0, sortOrder = 0;
+
+        int maxFieldPos = keyValueProviders.length / 2;
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            for (int k = 0; k < leftRTreeFrame.getTupleCount(); ++k) {
+
+                frameTuple.resetByTupleIndex(leftRTreeFrame, k);
+                double LowerKey = keyValueProviders[i]
+                        .getValue(frameTuple.getFieldData(i), frameTuple.getFieldStart(i));
+                double UpperKey = keyValueProviders[j]
+                        .getValue(frameTuple.getFieldData(j), frameTuple.getFieldStart(j));
+
+                tupleEntries1.add(k, LowerKey);
+                tupleEntries2.add(k, UpperKey);
+            }
+            double LowerKey = keyValueProviders[i].getValue(tuple.getFieldData(i), tuple.getFieldStart(i));
+            double UpperKey = keyValueProviders[j].getValue(tuple.getFieldData(j), tuple.getFieldStart(j));
+
+            tupleEntries1.add(-1, LowerKey);
+            tupleEntries2.add(-1, UpperKey);
+
+            tupleEntries1.sort(EntriesOrder.ASCENDING, leftRTreeFrame.getTupleCount() + 1);
+            tupleEntries2.sort(EntriesOrder.ASCENDING, leftRTreeFrame.getTupleCount() + 1);
+
+            double lowerMargin = 0.0, upperMargin = 0.0;
+            // generate distribution
+            for (int k = 1; k <= splitDistribution; ++k) {
+                int d = m - 1 + k;
+
+                generateDist(leftRTreeFrame, frameTuple, tuple, tupleEntries1, rec[0], 0, d);
+                generateDist(leftRTreeFrame, frameTuple, tuple, tupleEntries2, rec[1], 0, d);
+                generateDist(leftRTreeFrame, frameTuple, tuple, tupleEntries1, rec[2], d,
+                        leftRTreeFrame.getTupleCount() + 1);
+                generateDist(leftRTreeFrame, frameTuple, tuple, tupleEntries2, rec[3], d,
+                        leftRTreeFrame.getTupleCount() + 1);
+
+                // calculate the margin of the distributions
+                lowerMargin += rec[0].margin() + rec[2].margin();
+                upperMargin += rec[1].margin() + rec[3].margin();
+            }
+            double margin = Math.min(lowerMargin, upperMargin);
+
+            // store minimum margin as split axis
+            if (margin < minMargin) {
+                minMargin = margin;
+                splitAxis = i;
+                sortOrder = (lowerMargin < upperMargin) ? 0 : 2;
+            }
+
+            tupleEntries1.clear();
+            tupleEntries2.clear();
+        }
+
+        for (int i = 0; i < leftRTreeFrame.getTupleCount(); ++i) {
+            frameTuple.resetByTupleIndex(leftRTreeFrame, i);
+            double key = keyValueProviders[splitAxis + sortOrder].getValue(
+                    frameTuple.getFieldData(splitAxis + sortOrder), frameTuple.getFieldStart(splitAxis + sortOrder));
+            tupleEntries1.add(i, key);
+        }
+        double key = keyValueProviders[splitAxis + sortOrder].getValue(tuple.getFieldData(splitAxis + sortOrder),
+                tuple.getFieldStart(splitAxis + sortOrder));
+        tupleEntries1.add(-1, key);
+        tupleEntries1.sort(EntriesOrder.ASCENDING, leftRTreeFrame.getTupleCount() + 1);
+
+        double minArea = Double.MAX_VALUE;
+        double minOverlap = Double.MAX_VALUE;
+        int splitPoint = 0;
+        for (int i = 1; i <= splitDistribution; ++i) {
+            int d = m - 1 + i;
+
+            generateDist(leftRTreeFrame, frameTuple, tuple, tupleEntries1, rec[0], 0, d);
+            generateDist(leftRTreeFrame, frameTuple, tuple, tupleEntries1, rec[2], d,
+                    leftRTreeFrame.getTupleCount() + 1);
+
+            double overlap = rec[0].overlappedArea(rec[2]);
+            if (overlap < minOverlap) {
+                splitPoint = d;
+                minOverlap = overlap;
+                minArea = rec[0].area() + rec[2].area();
+            } else if (overlap == minOverlap) {
+                double area = rec[0].area() + rec[2].area();
+                if (area < minArea) {
+                    splitPoint = d;
+                    minArea = area;
+                }
+            }
+        }
+        int startIndex, endIndex;
+        if (splitPoint < (leftRTreeFrame.getTupleCount() + 1) / 2) {
+            startIndex = 0;
+            endIndex = splitPoint;
+        } else {
+            startIndex = splitPoint;
+            endIndex = (leftRTreeFrame.getTupleCount() + 1);
+        }
+        boolean insertedNewTupleInRightFrame = false;
+        int totalBytes = 0, numOfDeletedTuples = 0;
+        for (int i = startIndex; i < endIndex; i++) {
+            if (tupleEntries1.get(i).getTupleIndex() != -1) {
+                frameTuple.resetByTupleIndex(leftRTreeFrame, tupleEntries1.get(i).getTupleIndex());
+                rightFrame.insert(frameTuple, -1);
+                ((UnorderedSlotManager) slotManager).modifySlot(
+                        slotManager.getSlotOff(tupleEntries1.get(i).getTupleIndex()), -1);
+                totalBytes += leftRTreeFrame.getTupleSize(frameTuple);
+                numOfDeletedTuples++;
+            } else {
+                insertedNewTupleInRightFrame = true;
+            }
+        }
+
+        ((UnorderedSlotManager) slotManager).deleteEmptySlots();
+
+        // maintain space information
+        buf.putInt(totalFreeSpaceOff, buf.getInt(totalFreeSpaceOff) + totalBytes
+                + (slotManager.getSlotSize() * numOfDeletedTuples));
+
+        // compact both pages
+        rightFrame.compact();
+        leftRTreeFrame.compact();
+
+        // The assumption here is that the new tuple cannot be larger than page
+        // size, thus it must fit in either pages.
+        if (insertedNewTupleInRightFrame) {
+            if (rightFrame.hasSpaceInsert(tuple) == FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE) {
+                rightFrame.insert(tuple, -1);
+            } else {
+                leftRTreeFrame.insert(tuple, -1);
+            }
+        } else if (leftRTreeFrame.hasSpaceInsert(tuple) == FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE) {
+            leftRTreeFrame.insert(tuple, -1);
+        } else {
+            rightFrame.insert(tuple, -1);
+        }
+
+        int tupleOff = slotManager.getTupleOff(slotManager.getSlotEndOff());
+        frameTuple.resetByTupleOffset(buf, tupleOff);
+        int splitKeySize = tupleWriter.bytesRequired(frameTuple, 0, keyValueProviders.length);
+
+        splitKey.initData(splitKeySize);
+        leftRTreeFrame.adjustMBR();
+        rTreeTupleWriterleftRTreeFrame.writeTupleFields(leftRTreeFrame.getTuples(), 0,
+                rTreeSplitKey.getLeftPageBuffer(), 0);
+        rTreeSplitKey.getLeftTuple().resetByTupleOffset(rTreeSplitKey.getLeftPageBuffer(), 0);
+
+        ((IRTreeFrame) rightFrame).adjustMBR();
+        rTreeTupleWriterRightFrame.writeTupleFields(((RTreeNSMFrame) rightFrame).getTuples(), 0,
+                rTreeSplitKey.getRightPageBuffer(), 0);
+        rTreeSplitKey.getRightTuple().resetByTupleOffset(rTreeSplitKey.getRightPageBuffer(), 0);
+
+        tupleEntries1.clear();
+        tupleEntries2.clear();
+    }
+
+    public void generateDist(ITreeIndexFrame leftRTreeFrame, ITreeIndexTupleReference frameTuple,
+            ITupleReference tuple, TupleEntryArrayList entries, Rectangle rec, int start, int end) {
+        int j = 0;
+        while (entries.get(j).getTupleIndex() == -1) {
+            j++;
+        }
+        frameTuple.resetByTupleIndex(leftRTreeFrame, entries.get(j).getTupleIndex());
+        rec.set(frameTuple, keyValueProviders);
+        for (int i = start; i < end; ++i) {
+            if (i != j) {
+                if (entries.get(i).getTupleIndex() != -1) {
+                    frameTuple.resetByTupleIndex(leftRTreeFrame, entries.get(i).getTupleIndex());
+                    rec.enlarge(frameTuple, keyValueProviders);
+                } else {
+                    rec.enlarge(tuple, keyValueProviders);
+                }
+            }
+        }
+    }
+
+    @Override
+    public int findBestChildPosition(ITreeIndexFrame frame, ITupleReference tuple, ITreeIndexTupleReference frameTuple,
+            MultiComparator cmp) {
+        cmpFrameTuple.setFieldCount(cmp.getKeyFieldCount());
+        frameTuple.setFieldCount(cmp.getKeyFieldCount());
+
+        int bestChild = 0;
+        double minEnlargedArea = Double.MAX_VALUE;
+
+        // the children pointers in the node point to leaves
+        if (frame.getLevel() == 1) {
+            // find least overlap enlargement, use minimum enlarged area to
+            // break tie, if tie still exists use minimum area to break it
+            for (int i = 0; i < frame.getTupleCount(); ++i) {
+                frameTuple.resetByTupleIndex(frame, i);
+                double enlargedArea = RTreeComputationUtils.enlargedArea(frameTuple, tuple, cmp, keyValueProviders);
+                tupleEntries1.add(i, enlargedArea);
+                if (enlargedArea < minEnlargedArea) {
+                    minEnlargedArea = enlargedArea;
+                    bestChild = i;
+                }
+            }
+            if (minEnlargedArea < RTreeNSMFrame.doubleEpsilon() || minEnlargedArea > RTreeNSMFrame.doubleEpsilon()) {
+                minEnlargedArea = Double.MAX_VALUE;
+                int k;
+                if (frame.getTupleCount() > nearMinimumOverlapFactor) {
+                    // sort the entries based on their area enlargement needed
+                    // to include the object
+                    tupleEntries1.sort(EntriesOrder.ASCENDING, frame.getTupleCount());
+                    k = nearMinimumOverlapFactor;
+                } else {
+                    k = frame.getTupleCount();
+                }
+
+                double minOverlap = Double.MAX_VALUE;
+                int id = 0;
+                for (int i = 0; i < k; ++i) {
+                    double difference = 0.0;
+                    for (int j = 0; j < frame.getTupleCount(); ++j) {
+                        frameTuple.resetByTupleIndex(frame, j);
+                        cmpFrameTuple.resetByTupleIndex(frame, tupleEntries1.get(i).getTupleIndex());
+
+                        int c = ((RTreeNSMInteriorFrame) frame).pointerCmp(frameTuple, cmpFrameTuple, cmp);
+                        if (c != 0) {
+                            double intersection = RTreeComputationUtils.overlappedArea(frameTuple, tuple,
+                                    cmpFrameTuple, cmp, keyValueProviders);
+                            if (intersection != 0.0) {
+                                difference += intersection
+                                        - RTreeComputationUtils.overlappedArea(frameTuple, null, cmpFrameTuple, cmp,
+                                                keyValueProviders);
+                            }
+                        } else {
+                            id = j;
+                        }
+                    }
+
+                    double enlargedArea = RTreeComputationUtils.enlargedArea(cmpFrameTuple, tuple, cmp,
+                            keyValueProviders);
+                    if (difference < minOverlap) {
+                        minOverlap = difference;
+                        minEnlargedArea = enlargedArea;
+                        bestChild = id;
+                    } else if (difference == minOverlap) {
+                        if (enlargedArea < minEnlargedArea) {
+                            minEnlargedArea = enlargedArea;
+                            bestChild = id;
+                        } else if (enlargedArea == minEnlargedArea) {
+                            double area = RTreeComputationUtils.area(cmpFrameTuple, cmp, keyValueProviders);
+                            frameTuple.resetByTupleIndex(frame, bestChild);
+                            double minArea = RTreeComputationUtils.area(frameTuple, cmp, keyValueProviders);
+                            if (area < minArea) {
+                                bestChild = id;
+                            }
+                        }
+                    }
+                }
+            }
+        } else { // find minimum enlarged area, use minimum area to break tie
+            for (int i = 0; i < frame.getTupleCount(); i++) {
+                frameTuple.resetByTupleIndex(frame, i);
+                double enlargedArea = RTreeComputationUtils.enlargedArea(frameTuple, tuple, cmp, keyValueProviders);
+                if (enlargedArea < minEnlargedArea) {
+                    minEnlargedArea = enlargedArea;
+                    bestChild = i;
+                } else if (enlargedArea == minEnlargedArea) {
+                    double area = RTreeComputationUtils.area(frameTuple, cmp, keyValueProviders);
+                    frameTuple.resetByTupleIndex(frame, bestChild);
+                    double minArea = RTreeComputationUtils.area(frameTuple, cmp, keyValueProviders);
+                    if (area < minArea) {
+                        bestChild = i;
+                    }
+                }
+            }
+        }
+        tupleEntries1.clear();
+
+        return bestChild;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeComputationUtils.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeComputationUtils.java
new file mode 100644
index 0000000..f0122b3
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeComputationUtils.java
@@ -0,0 +1,137 @@
+/*
+ * Copyright 2009-2010 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.rtree.frames;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+
+public class RTreeComputationUtils {
+
+    public static double enlargedArea(ITupleReference tuple, ITupleReference tupleToBeInserted, MultiComparator cmp,
+            IPrimitiveValueProvider[] keyValueProviders) {
+        double areaBeforeEnlarge = RTreeComputationUtils.area(tuple, cmp, keyValueProviders);
+        double areaAfterEnlarge = 1.0;
+
+        int maxFieldPos = cmp.getKeyFieldCount() / 2;
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            double pHigh, pLow;
+            int c = cmp.getComparators()[i].compare(tuple.getFieldData(i), tuple.getFieldStart(i),
+                    tuple.getFieldLength(i), tupleToBeInserted.getFieldData(i), tupleToBeInserted.getFieldStart(i),
+                    tupleToBeInserted.getFieldLength(i));
+            if (c < 0) {
+                pLow = keyValueProviders[i].getValue(tuple.getFieldData(i), tuple.getFieldStart(i));
+            } else {
+                pLow = keyValueProviders[i].getValue(tupleToBeInserted.getFieldData(i),
+                        tupleToBeInserted.getFieldStart(i));
+            }
+
+            c = cmp.getComparators()[j].compare(tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j),
+                    tupleToBeInserted.getFieldData(j), tupleToBeInserted.getFieldStart(j),
+                    tupleToBeInserted.getFieldLength(j));
+            if (c > 0) {
+                pHigh = keyValueProviders[j].getValue(tuple.getFieldData(j), tuple.getFieldStart(j));
+            } else {
+                pHigh = keyValueProviders[j].getValue(tupleToBeInserted.getFieldData(j),
+                        tupleToBeInserted.getFieldStart(j));
+            }
+            areaAfterEnlarge *= pHigh - pLow;
+        }
+        return areaAfterEnlarge - areaBeforeEnlarge;
+    }
+
+    public static double overlappedArea(ITupleReference tuple1, ITupleReference tupleToBeInserted,
+            ITupleReference tuple2, MultiComparator cmp, IPrimitiveValueProvider[] keyValueProviders) {
+        double area = 1.0;
+        double f1, f2;
+
+        int maxFieldPos = cmp.getKeyFieldCount() / 2;
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            double pHigh1, pLow1;
+            if (tupleToBeInserted != null) {
+                int c = cmp.getComparators()[i].compare(tuple1.getFieldData(i), tuple1.getFieldStart(i),
+                        tuple1.getFieldLength(i), tupleToBeInserted.getFieldData(i),
+                        tupleToBeInserted.getFieldStart(i), tupleToBeInserted.getFieldLength(i));
+                if (c < 0) {
+                    pLow1 = keyValueProviders[i].getValue(tuple1.getFieldData(i), tuple1.getFieldStart(i));
+                } else {
+                    pLow1 = keyValueProviders[i].getValue(tupleToBeInserted.getFieldData(i),
+                            tupleToBeInserted.getFieldStart(i));
+                }
+
+                c = cmp.getComparators()[j].compare(tuple1.getFieldData(j), tuple1.getFieldStart(j),
+                        tuple1.getFieldLength(j), tupleToBeInserted.getFieldData(j),
+                        tupleToBeInserted.getFieldStart(j), tupleToBeInserted.getFieldLength(j));
+                if (c > 0) {
+                    pHigh1 = keyValueProviders[j].getValue(tuple1.getFieldData(j), tuple1.getFieldStart(j));
+                } else {
+                    pHigh1 = keyValueProviders[j].getValue(tupleToBeInserted.getFieldData(j),
+                            tupleToBeInserted.getFieldStart(j));
+                }
+            } else {
+                pLow1 = keyValueProviders[i].getValue(tuple1.getFieldData(i), tuple1.getFieldStart(i));
+                pHigh1 = keyValueProviders[j].getValue(tuple1.getFieldData(j), tuple1.getFieldStart(j));
+            }
+
+            double pLow2 = keyValueProviders[i].getValue(tuple2.getFieldData(i), tuple2.getFieldStart(i));
+            double pHigh2 = keyValueProviders[j].getValue(tuple2.getFieldData(j), tuple2.getFieldStart(j));
+
+            if (pLow1 > pHigh2 || pHigh1 < pLow2) {
+                return 0.0;
+            }
+
+            f1 = Math.max(pLow1, pLow2);
+            f2 = Math.min(pHigh1, pHigh2);
+            area *= f2 - f1;
+        }
+        return area;
+    }
+
+    public static double area(ITupleReference tuple, MultiComparator cmp, IPrimitiveValueProvider[] keyValueProviders) {
+        double area = 1.0;
+        int maxFieldPos = cmp.getKeyFieldCount() / 2;
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            area *= keyValueProviders[j].getValue(tuple.getFieldData(j), tuple.getFieldStart(j))
+                    - keyValueProviders[i].getValue(tuple.getFieldData(i), tuple.getFieldStart(i));
+        }
+        return area;
+    }
+
+    public static boolean containsRegion(ITupleReference tuple1, ITupleReference tuple2, MultiComparator cmp,
+            IPrimitiveValueProvider[] keyValueProviders) {
+        int maxFieldPos = cmp.getKeyFieldCount() / 2;
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            int c = cmp.getComparators()[i]
+                    .compare(tuple1.getFieldData(i), tuple1.getFieldStart(i), tuple1.getFieldLength(i),
+                            tuple2.getFieldData(i), tuple2.getFieldStart(i), tuple2.getFieldLength(i));
+            if (c > 0) {
+                return false;
+            }
+
+            c = cmp.getComparators()[j]
+                    .compare(tuple1.getFieldData(j), tuple1.getFieldStart(j), tuple1.getFieldLength(j),
+                            tuple2.getFieldData(j), tuple2.getFieldStart(j), tuple2.getFieldLength(j));
+            if (c < 0) {
+                return false;
+            }
+        }
+        return true;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java
index 84e66ef..eeada0a 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java
@@ -21,15 +21,11 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.frames.TreeIndexNSMFrame;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrame;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.EntriesOrder;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSplitKey;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.Rectangle;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.TupleEntryArrayList;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreePolicy;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.UnorderedSlotManager;
-import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
 
 public abstract class RTreeNSMFrame extends TreeIndexNSMFrame implements IRTreeFrame {
     protected static final int pageNsnOff = smFlagOff + 1;
@@ -37,33 +33,27 @@
 
     protected ITreeIndexTupleReference[] tuples;
     protected ITreeIndexTupleReference cmpFrameTuple;
-    protected TupleEntryArrayList tupleEntries1; // used for split and checking
-                                                 // enlargement
-    protected TupleEntryArrayList tupleEntries2; // used for split
 
-    protected Rectangle[] rec;
-
-    protected static final double splitFactor = 0.4;
-    protected static final int nearMinimumOverlapFactor = 32;
     private static final double doubleEpsilon = computeDoubleEpsilon();
-    private static final int numTuplesEntries = 100;
     protected final IPrimitiveValueProvider[] keyValueProviders;
 
-    public RTreeNSMFrame(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders) {
+    protected IRTreePolicy rtreePolicy;
+
+    public RTreeNSMFrame(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders,
+            RTreePolicyType rtreePolicyType) {
         super(tupleWriter, new UnorderedSlotManager());
         this.tuples = new ITreeIndexTupleReference[keyValueProviders.length];
         for (int i = 0; i < keyValueProviders.length; i++) {
             this.tuples[i] = tupleWriter.createTupleReference();
         }
         cmpFrameTuple = tupleWriter.createTupleReference();
-
-        tupleEntries1 = new TupleEntryArrayList(numTuplesEntries, numTuplesEntries);
-        tupleEntries2 = new TupleEntryArrayList(numTuplesEntries, numTuplesEntries);
-        rec = new Rectangle[4];
-        for (int i = 0; i < 4; i++) {
-            rec[i] = new Rectangle(keyValueProviders.length / 2);
-        }
         this.keyValueProviders = keyValueProviders;
+
+        if (rtreePolicyType == RTreePolicyType.RTREE) {
+            rtreePolicy = new RTreePolicy(tupleWriter, keyValueProviders, cmpFrameTuple, totalFreeSpaceOff);
+        } else {
+            rtreePolicy = new RStarTreePolicy(tupleWriter, keyValueProviders, cmpFrameTuple, totalFreeSpaceOff);
+        }
     }
 
     private static double computeDoubleEpsilon() {
@@ -116,184 +106,17 @@
         buf.putInt(rightPageOff, rightPage);
     }
 
-    protected ITreeIndexTupleReference[] getTuples() {
+    public ITreeIndexTupleReference[] getTuples() {
         return tuples;
     }
 
     @Override
-    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) throws TreeIndexException {
-        RTreeSplitKey rTreeSplitKey = ((RTreeSplitKey) splitKey);
-        RTreeTypeAwareTupleWriter rTreeTupleWriterLeftFrame = ((RTreeTypeAwareTupleWriter) tupleWriter);
-        RTreeTypeAwareTupleWriter rTreeTupleWriterRightFrame = ((RTreeTypeAwareTupleWriter) rightFrame.getTupleWriter());
-
-        // calculations are based on the R*-tree paper
-        int m = (int) Math.floor((getTupleCount() + 1) * splitFactor);
-        int splitDistribution = getTupleCount() - (2 * m) + 2;
-
-        // to calculate the minimum margin in order to pick the split axis
-        double minMargin = Double.MAX_VALUE;
-        int splitAxis = 0, sortOrder = 0;
-
-        int maxFieldPos = keyValueProviders.length / 2;
-        for (int i = 0; i < maxFieldPos; i++) {
-            int j = maxFieldPos + i;
-            for (int k = 0; k < getTupleCount(); ++k) {
-
-                frameTuple.resetByTupleIndex(this, k);
-                double LowerKey = keyValueProviders[i]
-                        .getValue(frameTuple.getFieldData(i), frameTuple.getFieldStart(i));
-                double UpperKey = keyValueProviders[j]
-                        .getValue(frameTuple.getFieldData(j), frameTuple.getFieldStart(j));
-
-                tupleEntries1.add(k, LowerKey);
-                tupleEntries2.add(k, UpperKey);
-            }
-            double LowerKey = keyValueProviders[i].getValue(tuple.getFieldData(i), tuple.getFieldStart(i));
-            double UpperKey = keyValueProviders[j].getValue(tuple.getFieldData(j), tuple.getFieldStart(j));
-
-            tupleEntries1.add(-1, LowerKey);
-            tupleEntries2.add(-1, UpperKey);
-
-            tupleEntries1.sort(EntriesOrder.ASCENDING, getTupleCount() + 1);
-            tupleEntries2.sort(EntriesOrder.ASCENDING, getTupleCount() + 1);
-
-            double lowerMargin = 0.0, upperMargin = 0.0;
-            // generate distribution
-            for (int k = 1; k <= splitDistribution; ++k) {
-                int d = m - 1 + k;
-
-                generateDist(tuple, tupleEntries1, rec[0], 0, d);
-                generateDist(tuple, tupleEntries2, rec[1], 0, d);
-                generateDist(tuple, tupleEntries1, rec[2], d, getTupleCount() + 1);
-                generateDist(tuple, tupleEntries2, rec[3], d, getTupleCount() + 1);
-
-                // calculate the margin of the distributions
-                lowerMargin += rec[0].margin() + rec[2].margin();
-                upperMargin += rec[1].margin() + rec[3].margin();
-            }
-            double margin = Math.min(lowerMargin, upperMargin);
-
-            // store minimum margin as split axis
-            if (margin < minMargin) {
-                minMargin = margin;
-                splitAxis = i;
-                sortOrder = (lowerMargin < upperMargin) ? 0 : 2;
-            }
-
-            tupleEntries1.clear();
-            tupleEntries2.clear();
-        }
-
-        for (int i = 0; i < getTupleCount(); ++i) {
-            frameTuple.resetByTupleIndex(this, i);
-            double key = keyValueProviders[splitAxis + sortOrder].getValue(
-                    frameTuple.getFieldData(splitAxis + sortOrder), frameTuple.getFieldStart(splitAxis + sortOrder));
-            tupleEntries1.add(i, key);
-        }
-        double key = keyValueProviders[splitAxis + sortOrder].getValue(tuple.getFieldData(splitAxis + sortOrder),
-                tuple.getFieldStart(splitAxis + sortOrder));
-        tupleEntries1.add(-1, key);
-        tupleEntries1.sort(EntriesOrder.ASCENDING, getTupleCount() + 1);
-
-        double minArea = Double.MAX_VALUE;
-        double minOverlap = Double.MAX_VALUE;
-        int splitPoint = 0;
-        for (int i = 1; i <= splitDistribution; ++i) {
-            int d = m - 1 + i;
-
-            generateDist(tuple, tupleEntries1, rec[0], 0, d);
-            generateDist(tuple, tupleEntries1, rec[2], d, getTupleCount() + 1);
-
-            double overlap = rec[0].overlappedArea(rec[2]);
-            if (overlap < minOverlap) {
-                splitPoint = d;
-                minOverlap = overlap;
-                minArea = rec[0].area() + rec[2].area();
-            } else if (overlap == minOverlap) {
-                double area = rec[0].area() + rec[2].area();
-                if (area < minArea) {
-                    splitPoint = d;
-                    minArea = area;
-                }
-            }
-        }
-        int startIndex, endIndex;
-        if (splitPoint < (getTupleCount() + 1) / 2) {
-            startIndex = 0;
-            endIndex = splitPoint;
-        } else {
-            startIndex = splitPoint;
-            endIndex = (getTupleCount() + 1);
-        }
-        boolean tupleInserted = false;
-        int totalBytes = 0, numOfDeletedTuples = 0;
-        for (int i = startIndex; i < endIndex; i++) {
-            if (tupleEntries1.get(i).getTupleIndex() != -1) {
-                frameTuple.resetByTupleIndex(this, tupleEntries1.get(i).getTupleIndex());
-                rightFrame.insert(frameTuple, -1);
-                ((UnorderedSlotManager) slotManager).modifySlot(
-                        slotManager.getSlotOff(tupleEntries1.get(i).getTupleIndex()), -1);
-                totalBytes += getTupleSize(frameTuple);
-                numOfDeletedTuples++;
-            } else {
-                rightFrame.insert(tuple, -1);
-                tupleInserted = true;
-            }
-        }
-
-        ((UnorderedSlotManager) slotManager).deleteEmptySlots();
-
-        // maintain space information
-        buf.putInt(totalFreeSpaceOff, buf.getInt(totalFreeSpaceOff) + totalBytes
-                + (slotManager.getSlotSize() * numOfDeletedTuples));
-
-        // compact both pages
-        rightFrame.compact();
-        compact();
-
-        if (!tupleInserted) {
-            insert(tuple, -1);
-        }
-
-        int tupleOff = slotManager.getTupleOff(slotManager.getSlotEndOff());
-        frameTuple.resetByTupleOffset(buf, tupleOff);
-        int splitKeySize = tupleWriter.bytesRequired(frameTuple, 0, keyValueProviders.length);
-
-        splitKey.initData(splitKeySize);
-        this.adjustMBR();
-        rTreeTupleWriterLeftFrame.writeTupleFields(getTuples(), 0, rTreeSplitKey.getLeftPageBuffer(), 0);
-        rTreeSplitKey.getLeftTuple().resetByTupleOffset(rTreeSplitKey.getLeftPageBuffer(), 0);
-
-        ((IRTreeFrame) rightFrame).adjustMBR();
-        rTreeTupleWriterRightFrame.writeTupleFields(((RTreeNSMFrame) rightFrame).getTuples(), 0,
-                rTreeSplitKey.getRightPageBuffer(), 0);
-        rTreeSplitKey.getRightTuple().resetByTupleOffset(rTreeSplitKey.getRightPageBuffer(), 0);
-
-        tupleEntries1.clear();
-        tupleEntries2.clear();
+    public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey splitKey) {
+        rtreePolicy.split(this, buf, rightFrame, slotManager, frameTuple, tuple, splitKey);
     }
 
     abstract public int getTupleSize(ITupleReference tuple);
 
-    public void generateDist(ITupleReference tuple, TupleEntryArrayList entries, Rectangle rec, int start, int end) {
-        int j = 0;
-        while (entries.get(j).getTupleIndex() == -1) {
-            j++;
-        }
-        frameTuple.resetByTupleIndex(this, entries.get(j).getTupleIndex());
-        rec.set(frameTuple, keyValueProviders);
-        for (int i = start; i < end; ++i) {
-            if (i != j) {
-                if (entries.get(i).getTupleIndex() != -1) {
-                    frameTuple.resetByTupleIndex(this, entries.get(i).getTupleIndex());
-                    rec.enlarge(frameTuple, keyValueProviders);
-                } else {
-                    rec.enlarge(tuple, keyValueProviders);
-                }
-            }
-        }
-    }
-
     public void adjustMBRImpl(ITreeIndexTupleReference[] tuples) {
         int maxFieldPos = keyValueProviders.length / 2;
         for (int i = 1; i < getTupleCount(); i++) {
@@ -328,6 +151,11 @@
 
     @Override
     public int getPageHeaderSize() {
-        return rightPageOff;
+        return rightPageOff + 4;
+    }
+
+    @Override
+    public void setMultiComparator(MultiComparator cmp) {
+        // currently, R-Tree Frames are unsorted
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
index 63387ef..5ab9632 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
@@ -31,7 +31,6 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.SlotOffTupleOff;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.EntriesOrder;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.PathList;
 
 public class RTreeNSMInteriorFrame extends RTreeNSMFrame implements IRTreeInteriorFrame {
@@ -41,109 +40,24 @@
             .createBinaryComparator();
     private final int keyFieldCount;
 
-    public RTreeNSMInteriorFrame(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders) {
-        super(tupleWriter, keyValueProviders);
+    public RTreeNSMInteriorFrame(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders,
+            RTreePolicyType rtreePolicyType) {
+        super(tupleWriter, keyValueProviders, rtreePolicyType);
         keyFieldCount = keyValueProviders.length;
         frameTuple.setFieldCount(keyFieldCount);
     }
 
     @Override
-    public boolean findBestChild(ITupleReference tuple, MultiComparator cmp) {
-        cmpFrameTuple.setFieldCount(cmp.getKeyFieldCount());
-        frameTuple.setFieldCount(cmp.getKeyFieldCount());
-
-        int bestChild = 0;
-        double minEnlargedArea = Double.MAX_VALUE;
-
-        // the children pointers in the node point to leaves
-        if (getLevel() == 1) {
-            // find least overlap enlargement, use minimum enlarged area to
-            // break tie, if tie still exists use minimum area to break it
-            for (int i = 0; i < getTupleCount(); ++i) {
-                frameTuple.resetByTupleIndex(this, i);
-                double enlargedArea = enlargedArea(frameTuple, tuple, cmp);
-                tupleEntries1.add(i, enlargedArea);
-                if (enlargedArea < minEnlargedArea) {
-                    minEnlargedArea = enlargedArea;
-                    bestChild = i;
-                }
-            }
-            if (minEnlargedArea < RTreeNSMFrame.doubleEpsilon() || minEnlargedArea > RTreeNSMFrame.doubleEpsilon()) {
-                minEnlargedArea = Double.MAX_VALUE;
-                int k;
-                if (getTupleCount() > nearMinimumOverlapFactor) {
-                    // sort the entries based on their area enlargement needed
-                    // to include the object
-                    tupleEntries1.sort(EntriesOrder.ASCENDING, getTupleCount());
-                    k = nearMinimumOverlapFactor;
-                } else {
-                    k = getTupleCount();
-                }
-
-                double minOverlap = Double.MAX_VALUE;
-                int id = 0;
-                for (int i = 0; i < k; ++i) {
-                    double difference = 0.0;
-                    for (int j = 0; j < getTupleCount(); ++j) {
-                        frameTuple.resetByTupleIndex(this, j);
-                        cmpFrameTuple.resetByTupleIndex(this, tupleEntries1.get(i).getTupleIndex());
-
-                        int c = pointerCmp(frameTuple, cmpFrameTuple, cmp);
-                        if (c != 0) {
-                            double intersection = overlappedArea(frameTuple, tuple, cmpFrameTuple, cmp);
-                            if (intersection != 0.0) {
-                                difference += intersection - overlappedArea(frameTuple, null, cmpFrameTuple, cmp);
-                            }
-                        } else {
-                            id = j;
-                        }
-                    }
-
-                    double enlargedArea = enlargedArea(cmpFrameTuple, tuple, cmp);
-                    if (difference < minOverlap) {
-                        minOverlap = difference;
-                        minEnlargedArea = enlargedArea;
-                        bestChild = id;
-                    } else if (difference == minOverlap) {
-                        if (enlargedArea < minEnlargedArea) {
-                            minEnlargedArea = enlargedArea;
-                            bestChild = id;
-                        } else if (enlargedArea == minEnlargedArea) {
-                            double area = area(cmpFrameTuple, cmp);
-                            frameTuple.resetByTupleIndex(this, bestChild);
-                            double minArea = area(frameTuple, cmp);
-                            if (area < minArea) {
-                                bestChild = id;
-                            }
-                        }
-                    }
-                }
-            }
-        } else { // find minimum enlarged area, use minimum area to break tie
-            for (int i = 0; i < getTupleCount(); i++) {
-                frameTuple.resetByTupleIndex(this, i);
-                double enlargedArea = enlargedArea(frameTuple, tuple, cmp);
-                if (enlargedArea < minEnlargedArea) {
-                    minEnlargedArea = enlargedArea;
-                    bestChild = i;
-                } else if (enlargedArea == minEnlargedArea) {
-                    double area = area(frameTuple, cmp);
-                    frameTuple.resetByTupleIndex(this, bestChild);
-                    double minArea = area(frameTuple, cmp);
-                    if (area < minArea) {
-                        bestChild = i;
-                    }
-                }
-            }
-        }
-        tupleEntries1.clear();
-
+    public int findBestChild(ITupleReference tuple, MultiComparator cmp) {
+        int bestChild = rtreePolicy.findBestChildPosition(this, tuple, frameTuple, cmp);
         frameTuple.resetByTupleIndex(this, bestChild);
-        if (minEnlargedArea > 0.0) {
-            return true;
-        } else {
-            return false;
-        }
+        return buf.getInt(getChildPointerOff(frameTuple));
+    }
+
+    // frameTuple is assumed to have the tuple to be tested against.
+    @Override
+    public boolean checkIfEnlarementIsNeeded(ITupleReference tuple, MultiComparator cmp) {
+        return !RTreeComputationUtils.containsRegion(frameTuple, tuple, cmp, keyValueProviders);
     }
 
     @Override
@@ -154,11 +68,6 @@
     }
 
     @Override
-    public int getBestChildPageId() {
-        return buf.getInt(getChildPointerOff(frameTuple));
-    }
-
-    @Override
     public int findTupleByPointer(ITupleReference tuple, MultiComparator cmp) {
         frameTuple.setFieldCount(cmp.getKeyFieldCount());
         for (int i = 0; i < getTupleCount(); i++) {
@@ -279,7 +188,7 @@
 
     }
 
-    private int pointerCmp(ITupleReference tupleA, ITupleReference tupleB, MultiComparator cmp) {
+    protected int pointerCmp(ITupleReference tupleA, ITupleReference tupleB, MultiComparator cmp) {
         return childPtrCmp
                 .compare(tupleA.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleA), childPtrSize,
                         tupleB.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleB), childPtrSize);
@@ -330,143 +239,6 @@
     }
 
     @Override
-    public boolean recomputeMBR(ITupleReference tuple, int tupleIndex, MultiComparator cmp) {
-        frameTuple.setFieldCount(cmp.getKeyFieldCount());
-        frameTuple.resetByTupleIndex(this, tupleIndex);
-
-        int maxFieldPos = cmp.getKeyFieldCount() / 2;
-        for (int i = 0; i < maxFieldPos; i++) {
-            int j = maxFieldPos + i;
-            int c = cmp.getComparators()[i].compare(frameTuple.getFieldData(i), frameTuple.getFieldStart(i),
-                    frameTuple.getFieldLength(i), tuple.getFieldData(i), tuple.getFieldStart(i),
-                    tuple.getFieldLength(i));
-            if (c != 0) {
-                return true;
-            }
-            c = cmp.getComparators()[j].compare(frameTuple.getFieldData(j), frameTuple.getFieldStart(j),
-                    frameTuple.getFieldLength(j), tuple.getFieldData(j), tuple.getFieldStart(j),
-                    tuple.getFieldLength(j));
-
-            if (c != 0) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    private double overlappedArea(ITupleReference tuple1, ITupleReference tupleToBeInserted, ITupleReference tuple2,
-            MultiComparator cmp) {
-        double area = 1.0;
-        double f1, f2;
-
-        int maxFieldPos = cmp.getKeyFieldCount() / 2;
-        for (int i = 0; i < maxFieldPos; i++) {
-            int j = maxFieldPos + i;
-            double pHigh1, pLow1;
-            if (tupleToBeInserted != null) {
-                int c = cmp.getComparators()[i].compare(tuple1.getFieldData(i), tuple1.getFieldStart(i),
-                        tuple1.getFieldLength(i), tupleToBeInserted.getFieldData(i),
-                        tupleToBeInserted.getFieldStart(i), tupleToBeInserted.getFieldLength(i));
-                if (c < 0) {
-                    pLow1 = keyValueProviders[i].getValue(tuple1.getFieldData(i), tuple1.getFieldStart(i));
-                } else {
-                    pLow1 = keyValueProviders[i].getValue(tupleToBeInserted.getFieldData(i),
-                            tupleToBeInserted.getFieldStart(i));
-                }
-
-                c = cmp.getComparators()[j].compare(tuple1.getFieldData(j), tuple1.getFieldStart(j),
-                        tuple1.getFieldLength(j), tupleToBeInserted.getFieldData(j),
-                        tupleToBeInserted.getFieldStart(j), tupleToBeInserted.getFieldLength(j));
-                if (c > 0) {
-                    pHigh1 = keyValueProviders[j].getValue(tuple1.getFieldData(j), tuple1.getFieldStart(j));
-                } else {
-                    pHigh1 = keyValueProviders[j].getValue(tupleToBeInserted.getFieldData(j),
-                            tupleToBeInserted.getFieldStart(j));
-                }
-            } else {
-                pLow1 = keyValueProviders[i].getValue(tuple1.getFieldData(i), tuple1.getFieldStart(i));
-                pHigh1 = keyValueProviders[j].getValue(tuple1.getFieldData(j), tuple1.getFieldStart(j));
-            }
-
-            double pLow2 = keyValueProviders[i].getValue(tuple2.getFieldData(i), tuple2.getFieldStart(i));
-            double pHigh2 = keyValueProviders[j].getValue(tuple2.getFieldData(j), tuple2.getFieldStart(j));
-
-            if (pLow1 > pHigh2 || pHigh1 < pLow2) {
-                return 0.0;
-            }
-
-            f1 = Math.max(pLow1, pLow2);
-            f2 = Math.min(pHigh1, pHigh2);
-            area *= f2 - f1;
-        }
-        return area;
-    }
-
-    private double enlargedArea(ITupleReference tuple, ITupleReference tupleToBeInserted, MultiComparator cmp) {
-        double areaBeforeEnlarge = area(tuple, cmp);
-        double areaAfterEnlarge = 1.0;
-
-        int maxFieldPos = cmp.getKeyFieldCount() / 2;
-        for (int i = 0; i < maxFieldPos; i++) {
-            int j = maxFieldPos + i;
-            double pHigh, pLow;
-            int c = cmp.getComparators()[i].compare(tuple.getFieldData(i), tuple.getFieldStart(i),
-                    tuple.getFieldLength(i), tupleToBeInserted.getFieldData(i), tupleToBeInserted.getFieldStart(i),
-                    tupleToBeInserted.getFieldLength(i));
-            if (c < 0) {
-                pLow = keyValueProviders[i].getValue(tuple.getFieldData(i), tuple.getFieldStart(i));
-            } else {
-                pLow = keyValueProviders[i].getValue(tupleToBeInserted.getFieldData(i),
-                        tupleToBeInserted.getFieldStart(i));
-            }
-
-            c = cmp.getComparators()[j].compare(tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j),
-                    tupleToBeInserted.getFieldData(j), tupleToBeInserted.getFieldStart(j),
-                    tupleToBeInserted.getFieldLength(j));
-            if (c > 0) {
-                pHigh = keyValueProviders[j].getValue(tuple.getFieldData(j), tuple.getFieldStart(j));
-            } else {
-                pHigh = keyValueProviders[j].getValue(tupleToBeInserted.getFieldData(j),
-                        tupleToBeInserted.getFieldStart(j));
-            }
-            areaAfterEnlarge *= pHigh - pLow;
-        }
-        return areaAfterEnlarge - areaBeforeEnlarge;
-    }
-
-    private double area(ITupleReference tuple, MultiComparator cmp) {
-        double area = 1.0;
-        int maxFieldPos = cmp.getKeyFieldCount() / 2;
-        for (int i = 0; i < maxFieldPos; i++) {
-            int j = maxFieldPos + i;
-            area *= keyValueProviders[j].getValue(tuple.getFieldData(j), tuple.getFieldStart(j))
-                    - keyValueProviders[i].getValue(tuple.getFieldData(i), tuple.getFieldStart(i));
-        }
-        return area;
-    }
-
-    @Override
-    public boolean checkEnlargement(ITupleReference tuple, MultiComparator cmp) {
-        int maxFieldPos = cmp.getKeyFieldCount() / 2;
-        for (int i = 0; i < maxFieldPos; i++) {
-            int j = maxFieldPos + i;
-            int c = cmp.getComparators()[i].compare(frameTuple.getFieldData(i), frameTuple.getFieldStart(i),
-                    frameTuple.getFieldLength(i), tuple.getFieldData(i), tuple.getFieldStart(i),
-                    tuple.getFieldLength(i));
-            if (c > 0) {
-                return true;
-            }
-            c = cmp.getComparators()[j].compare(frameTuple.getFieldData(j), frameTuple.getFieldStart(j),
-                    frameTuple.getFieldLength(j), tuple.getFieldData(j), tuple.getFieldStart(j),
-                    tuple.getFieldLength(j));
-            if (c < 0) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @Override
     public void enlarge(ITupleReference tuple, MultiComparator cmp) {
         int maxFieldPos = cmp.getKeyFieldCount() / 2;
         for (int i = 0; i < maxFieldPos; i++) {
@@ -509,4 +281,8 @@
     public int getFieldCount() {
         return keyValueProviders.length;
     }
+
+    public int getChildPointerSize() {
+        return childPtrSize;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrameFactory.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrameFactory.java
index 943a179..fdb0e0a 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrameFactory.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrameFactory.java
@@ -26,13 +26,16 @@
     private static final long serialVersionUID = 1L;
     private final ITreeIndexTupleWriterFactory tupleWriterFactory;
     private final IPrimitiveValueProviderFactory[] keyValueProviderFactories;
+    private final RTreePolicyType rtreePolicyType;
 
-    public RTreeNSMInteriorFrameFactory(ITreeIndexTupleWriterFactory tupleWriterFactory, IPrimitiveValueProviderFactory[] keyValueProviderFactories) {
+    public RTreeNSMInteriorFrameFactory(ITreeIndexTupleWriterFactory tupleWriterFactory,
+            IPrimitiveValueProviderFactory[] keyValueProviderFactories, RTreePolicyType rtreePolicyType) {
         this.tupleWriterFactory = tupleWriterFactory;
         if (keyValueProviderFactories.length % 2 != 0) {
             throw new IllegalArgumentException("The key has different number of dimensions.");
         }
         this.keyValueProviderFactories = keyValueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
     }
 
     @Override
@@ -41,11 +44,11 @@
         for (int i = 0; i < keyValueProviders.length; i++) {
             keyValueProviders[i] = keyValueProviderFactories[i].createPrimitiveValueProvider();
         }
-        return new RTreeNSMInteriorFrame(tupleWriterFactory.createTupleWriter(), keyValueProviders);
+        return new RTreeNSMInteriorFrame(tupleWriterFactory.createTupleWriter(), keyValueProviders, rtreePolicyType);
     }
 
-	@Override
-	public ITreeIndexTupleWriterFactory getTupleWriterFactory() {
-		return tupleWriterFactory;
-	}
+    @Override
+    public ITreeIndexTupleWriterFactory getTupleWriterFactory() {
+        return tupleWriterFactory;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java
index f1d71ff..d52ef16 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java
@@ -24,8 +24,9 @@
 
 public class RTreeNSMLeafFrame extends RTreeNSMFrame implements IRTreeLeafFrame {
 
-    public RTreeNSMLeafFrame(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders) {
-        super(tupleWriter, keyValueProviders);
+    public RTreeNSMLeafFrame(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders,
+            RTreePolicyType rtreePolicyType) {
+        super(tupleWriter, keyValueProviders, rtreePolicyType);
     }
 
     @Override
@@ -96,4 +97,19 @@
     public int getFieldCount() {
         return frameTuple.getFieldCount();
     }
+
+    public ITupleReference getBeforeTuple(ITupleReference tuple, int targetTupleIndex, MultiComparator cmp) {
+        // Examine the tuple index to determine whether it is valid or not.
+        if (targetTupleIndex != slotManager.getGreatestKeyIndicator()) {
+            // We need to check the key to determine whether it's an insert or an update.
+            frameTuple.resetByTupleIndex(this, targetTupleIndex);
+            if (cmp.compare(tuple, frameTuple) == 0) {
+                // The keys match, it's an update.
+                return frameTuple;
+            }
+        }
+        // Either the tuple index is a special indicator, or the keys don't match.
+        // In those cases, we are definitely dealing with an insert.
+        return null;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrameFactory.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrameFactory.java
index e31148f..b4d382b 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrameFactory.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrameFactory.java
@@ -26,13 +26,16 @@
     private static final long serialVersionUID = 1L;
     private final ITreeIndexTupleWriterFactory tupleWriterFactory;
     private final IPrimitiveValueProviderFactory[] keyValueProviderFactories;
+    private final RTreePolicyType rtreePolicyType;
 
-    public RTreeNSMLeafFrameFactory(ITreeIndexTupleWriterFactory tupleWriterFactory, IPrimitiveValueProviderFactory[] keyValueProviderFactories) {
+    public RTreeNSMLeafFrameFactory(ITreeIndexTupleWriterFactory tupleWriterFactory,
+            IPrimitiveValueProviderFactory[] keyValueProviderFactories, RTreePolicyType rtreePolicyType) {
         this.tupleWriterFactory = tupleWriterFactory;
         if (keyValueProviderFactories.length % 2 != 0) {
             throw new IllegalArgumentException("The key has different number of dimensions.");
         }
         this.keyValueProviderFactories = keyValueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
     }
 
     @Override
@@ -41,11 +44,11 @@
         for (int i = 0; i < keyValueProviders.length; i++) {
             keyValueProviders[i] = keyValueProviderFactories[i].createPrimitiveValueProvider();
         }
-        return new RTreeNSMLeafFrame(tupleWriterFactory.createTupleWriter(), keyValueProviders);
+        return new RTreeNSMLeafFrame(tupleWriterFactory.createTupleWriter(), keyValueProviders, rtreePolicyType);
     }
 
-	@Override
-	public ITreeIndexTupleWriterFactory getTupleWriterFactory() {
-		return tupleWriterFactory;
-	}
+    @Override
+    public ITreeIndexTupleWriterFactory getTupleWriterFactory() {
+        return tupleWriterFactory;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreePolicy.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreePolicy.java
new file mode 100644
index 0000000..9d94794
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreePolicy.java
@@ -0,0 +1,232 @@
+/*
+ * Copyright 2009-2010 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.rtree.frames;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ISlotManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreePolicy;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSplitKey;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.Rectangle;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.UnorderedSlotManager;
+import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
+
+public class RTreePolicy implements IRTreePolicy {
+
+    private Rectangle[] rec;
+
+    private final ITreeIndexTupleWriter tupleWriter;
+    private final IPrimitiveValueProvider[] keyValueProviders;
+    private ITreeIndexTupleReference cmpFrameTuple;
+    private final int totalFreeSpaceOff;
+
+    public RTreePolicy(ITreeIndexTupleWriter tupleWriter, IPrimitiveValueProvider[] keyValueProviders,
+            ITreeIndexTupleReference cmpFrameTuple, int totalFreeSpaceOff) {
+        this.tupleWriter = tupleWriter;
+        this.keyValueProviders = keyValueProviders;
+        this.cmpFrameTuple = cmpFrameTuple;
+        this.totalFreeSpaceOff = totalFreeSpaceOff;
+
+        rec = new Rectangle[2];
+        for (int i = 0; i < 2; i++) {
+            rec[i] = new Rectangle(keyValueProviders.length / 2);
+        }
+    }
+
+    @Override
+    public void split(ITreeIndexFrame leftFrame, ByteBuffer buf, ITreeIndexFrame rightFrame, ISlotManager slotManager,
+            ITreeIndexTupleReference frameTuple, ITupleReference tuple, ISplitKey splitKey) {
+        RTreeSplitKey rTreeSplitKey = ((RTreeSplitKey) splitKey);
+        RTreeTypeAwareTupleWriter rTreeTupleWriterLeftFrame = ((RTreeTypeAwareTupleWriter) tupleWriter);
+        RTreeTypeAwareTupleWriter rTreeTupleWriterRightFrame = ((RTreeTypeAwareTupleWriter) rightFrame.getTupleWriter());
+
+        RTreeNSMFrame leftRTreeFrame = ((RTreeNSMFrame) leftFrame);
+
+        double separation = Double.NEGATIVE_INFINITY;
+        int seed1 = 0, seed2 = 0;
+        int maxFieldPos = keyValueProviders.length / 2;
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            frameTuple.resetByTupleIndex(leftRTreeFrame, 0);
+            double leastLowerValue = keyValueProviders[i].getValue(frameTuple.getFieldData(i),
+                    frameTuple.getFieldStart(i));
+            double greatestUpperValue = keyValueProviders[j].getValue(frameTuple.getFieldData(j),
+                    frameTuple.getFieldStart(j));
+            double leastUpperValue = leastLowerValue;
+            double greatestLowerValue = greatestUpperValue;
+            int leastUpperIndex = 0;
+            int greatestLowerIndex = 0;
+            double width;
+
+            int tupleCount = leftRTreeFrame.getTupleCount();
+            for (int k = 1; k < tupleCount; ++k) {
+                frameTuple.resetByTupleIndex(leftRTreeFrame, k);
+                double lowerValue = keyValueProviders[i].getValue(frameTuple.getFieldData(i),
+                        frameTuple.getFieldStart(i));
+                if (lowerValue > greatestLowerValue) {
+                    greatestLowerIndex = k;
+                    cmpFrameTuple.resetByTupleIndex(leftRTreeFrame, k);
+                    greatestLowerValue = keyValueProviders[i].getValue(cmpFrameTuple.getFieldData(i),
+                            cmpFrameTuple.getFieldStart(i));
+                }
+                double higherValue = keyValueProviders[j].getValue(frameTuple.getFieldData(j),
+                        frameTuple.getFieldStart(j));
+                if (higherValue < leastUpperValue) {
+                    leastUpperIndex = k;
+                    cmpFrameTuple.resetByTupleIndex(leftRTreeFrame, k);
+                    leastUpperValue = keyValueProviders[j].getValue(cmpFrameTuple.getFieldData(j),
+                            cmpFrameTuple.getFieldStart(j));
+                }
+
+                leastLowerValue = Math.min(lowerValue, leastLowerValue);
+                greatestUpperValue = Math.max(higherValue, greatestUpperValue);
+            }
+
+            width = greatestUpperValue - leastLowerValue;
+            if (width <= 0) {
+                width = 1;
+            }
+
+            double f = (greatestLowerValue - leastUpperValue) / width;
+
+            if (f > separation) {
+                seed1 = leastUpperIndex;
+                seed2 = greatestLowerIndex;
+                separation = f;
+            }
+        }
+
+        if (seed1 == seed2) {
+            if (seed1 == 0) {
+                seed2 = 1;
+            } else {
+                --seed2;
+            }
+        }
+
+        int totalBytes = 0, numOfDeletedTuples = 0;
+
+        frameTuple.resetByTupleIndex(leftRTreeFrame, seed1);
+        rec[0].set(frameTuple, keyValueProviders);
+        rightFrame.insert(frameTuple, -1);
+        ((UnorderedSlotManager) slotManager).modifySlot(slotManager.getSlotOff(seed1), -1);
+        totalBytes += leftRTreeFrame.getTupleSize(frameTuple);
+        numOfDeletedTuples++;
+
+        frameTuple.resetByTupleIndex(leftRTreeFrame, seed2);
+        rec[1].set(frameTuple, keyValueProviders);
+
+        int remainingTuplestoBeInsertedInRightFrame;
+        for (int k = 0; k < leftRTreeFrame.getTupleCount(); ++k) {
+            remainingTuplestoBeInsertedInRightFrame = leftRTreeFrame.getTupleCount() / 2 - rightFrame.getTupleCount();
+            if (remainingTuplestoBeInsertedInRightFrame == 0) {
+                break;
+            }
+            if (k != seed1 && k != seed2) {
+                frameTuple.resetByTupleIndex(leftRTreeFrame, k);
+                if (rec[0].enlargedArea(frameTuple, keyValueProviders) < rec[1].enlargedArea(frameTuple,
+                        keyValueProviders)
+                        || leftRTreeFrame.getTupleCount() - k <= remainingTuplestoBeInsertedInRightFrame) {
+                    rightFrame.insert(frameTuple, -1);
+                    rec[0].enlarge(frameTuple, keyValueProviders);
+                    ((UnorderedSlotManager) slotManager).modifySlot(slotManager.getSlotOff(k), -1);
+                    totalBytes += leftRTreeFrame.getTupleSize(frameTuple);
+                    numOfDeletedTuples++;
+                } else {
+                    rec[1].enlarge(frameTuple, keyValueProviders);
+                }
+            }
+
+        }
+
+        ((UnorderedSlotManager) slotManager).deleteEmptySlots();
+
+        // maintain space information
+        buf.putInt(totalFreeSpaceOff, buf.getInt(totalFreeSpaceOff) + totalBytes
+                + (slotManager.getSlotSize() * numOfDeletedTuples));
+
+        // compact both pages
+        rightFrame.compact();
+        leftRTreeFrame.compact();
+
+        // The assumption here is that the new tuple cannot be larger than page
+        // size, thus it must fit in either pages.
+        if (rec[0].enlargedArea(tuple, keyValueProviders) < rec[1].enlargedArea(tuple, keyValueProviders)) {
+            if (rightFrame.hasSpaceInsert(tuple) == FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE) {
+                rightFrame.insert(tuple, -1);
+            } else {
+                leftRTreeFrame.insert(tuple, -1);
+            }
+        } else if (leftRTreeFrame.hasSpaceInsert(tuple) == FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE) {
+            leftRTreeFrame.insert(tuple, -1);
+        } else {
+            rightFrame.insert(tuple, -1);
+        }
+
+        int tupleOff = slotManager.getTupleOff(slotManager.getSlotEndOff());
+        frameTuple.resetByTupleOffset(buf, tupleOff);
+        int splitKeySize = tupleWriter.bytesRequired(frameTuple, 0, keyValueProviders.length);
+
+        splitKey.initData(splitKeySize);
+        leftRTreeFrame.adjustMBR();
+        rTreeTupleWriterLeftFrame.writeTupleFields(leftRTreeFrame.getTuples(), 0, rTreeSplitKey.getLeftPageBuffer(), 0);
+        rTreeSplitKey.getLeftTuple().resetByTupleOffset(rTreeSplitKey.getLeftPageBuffer(), 0);
+
+        ((IRTreeFrame) rightFrame).adjustMBR();
+        rTreeTupleWriterRightFrame.writeTupleFields(((RTreeNSMFrame) rightFrame).getTuples(), 0,
+                rTreeSplitKey.getRightPageBuffer(), 0);
+        rTreeSplitKey.getRightTuple().resetByTupleOffset(rTreeSplitKey.getRightPageBuffer(), 0);
+    }
+
+    @Override
+    public int findBestChildPosition(ITreeIndexFrame frame, ITupleReference tuple, ITreeIndexTupleReference frameTuple,
+            MultiComparator cmp) {
+        cmpFrameTuple.setFieldCount(cmp.getKeyFieldCount());
+        frameTuple.setFieldCount(cmp.getKeyFieldCount());
+
+        int bestChild = 0;
+        double minEnlargedArea = Double.MAX_VALUE;
+
+        // find minimum enlarged area, use minimum area to break tie
+        for (int i = 0; i < frame.getTupleCount(); i++) {
+            frameTuple.resetByTupleIndex(frame, i);
+            double enlargedArea = RTreeComputationUtils.enlargedArea(frameTuple, tuple, cmp, keyValueProviders);
+            if (enlargedArea < minEnlargedArea) {
+                minEnlargedArea = enlargedArea;
+                bestChild = i;
+            } else if (enlargedArea == minEnlargedArea) {
+                double area = RTreeComputationUtils.area(frameTuple, cmp, keyValueProviders);
+                frameTuple.resetByTupleIndex(frame, bestChild);
+                double minArea = RTreeComputationUtils.area(frameTuple, cmp, keyValueProviders);
+                if (area < minArea) {
+                    bestChild = i;
+                }
+            }
+        }
+
+        return bestChild;
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreePolicyType.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreePolicyType.java
new file mode 100644
index 0000000..712c424
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreePolicyType.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2009-2010 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.rtree.frames;
+
+public enum RTreePolicyType {
+    RTREE, RSTARTREE
+}
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 cc3cf5b..c12dc50 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
@@ -15,88 +15,64 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree.impls;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.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.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.common.api.IndexType;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.impls.AbstractTreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.impls.NodeFrontier;
+import edu.uci.ics.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexUtils;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriter;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
-public class RTree implements ITreeIndex {
-
-    private final int rootPage = 1;
+public class RTree extends AbstractTreeIndex {
 
     // Global node sequence number used for the concurrency control protocol
     private final AtomicLong globalNsn;
-    private final ReadWriteLock treeLatch;
 
-    private final IFreePageManager freePageManager;
-    private final IBufferCache bufferCache;
-    private int fileId;
-
-    private final ITreeIndexFrameFactory interiorFrameFactory;
-    private final ITreeIndexFrameFactory leafFrameFactory;
-    private final int fieldCount;
-    private final IBinaryComparatorFactory[] cmpFactories;
-
-    public RTree(IBufferCache bufferCache, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
-            IFreePageManager freePageManager, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexFrameFactory leafFrameFactory) {
-        this.bufferCache = bufferCache;
-        this.fieldCount = fieldCount;
-        this.cmpFactories = cmpFactories;
-        this.freePageManager = freePageManager;
-        this.interiorFrameFactory = interiorFrameFactory;
-        this.leafFrameFactory = leafFrameFactory;
+    public RTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider, IFreePageManager freePageManager,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
+            IBinaryComparatorFactory[] cmpFactories, int fieldCount, FileReference file) {
+        super(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
+                fieldCount, file);
         globalNsn = new AtomicLong();
-        this.treeLatch = new ReentrantReadWriteLock(true);
     }
 
     private long incrementGlobalNsn() {
         return globalNsn.incrementAndGet();
     }
 
-    public byte getTreeHeight(IRTreeLeafFrame leafFrame) throws HyracksDataException {
-        ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false);
-        rootNode.acquireReadLatch();
-        try {
-            leafFrame.setPage(rootNode);
-            return leafFrame.getLevel();
-        } finally {
-            rootNode.releaseReadLatch();
-            bufferCache.unpin(rootNode);
-        }
-    }
-
     @SuppressWarnings("rawtypes")
     public String printTree(IRTreeLeafFrame leafFrame, IRTreeInteriorFrame interiorFrame,
             ISerializerDeserializer[] keySerdes) throws Exception {
@@ -162,66 +138,21 @@
         }
     }
 
-    @Override
-    public void create(int fileId) throws HyracksDataException {
-        treeLatch.writeLock().lock();
-        try {
-            ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
-            ITreeIndexMetaDataFrame metaFrame = freePageManager.getMetaDataFrameFactory().createFrame();
-            freePageManager.open(fileId);
-            freePageManager.init(metaFrame, rootPage);
-
-            // initialize root page
-            ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), true);
-
-            rootNode.acquireWriteLatch();
-            try {
-                leafFrame.setPage(rootNode);
-                leafFrame.initBuffer((byte) 0);
-            } finally {
-                rootNode.releaseWriteLatch();
-                bufferCache.unpin(rootNode);
-            }
-        } finally {
-            treeLatch.writeLock().unlock();
-        }
-    }
-
-    @Override
-    public void open(int fileId) {
-        this.fileId = fileId;
-        freePageManager.open(fileId);
-    }
-
-    @Override
-    public void close() {
-        fileId = -1;
-        freePageManager.close();
-    }
-
-    @Override
-    public int getFileId() {
-        return fileId;
-    }
-
-    @Override
-    public IBufferCache getBufferCache() {
-        return bufferCache;
-    }
-
-    private RTreeOpContext createOpContext() {
+    private RTreeOpContext createOpContext(IModificationOperationCallback modificationCallback) {
         return new RTreeOpContext((IRTreeLeafFrame) leafFrameFactory.createFrame(),
                 (IRTreeInteriorFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
-                        .createFrame(), cmpFactories, 8);
+                        .createFrame(), cmpFactories, 8, modificationCallback);
     }
 
-    private void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
+    private void insert(ITupleReference tuple, IIndexOperationContext ictx) throws HyracksDataException,
+            TreeIndexException {
         RTreeOpContext ctx = (RTreeOpContext) ictx;
         ctx.reset();
         ctx.setTuple(tuple);
         ctx.splitKey.reset();
         ctx.splitKey.getLeftTuple().setFieldCount(cmpFactories.length);
         ctx.splitKey.getRightTuple().setFieldCount(cmpFactories.length);
+        ctx.modificationCallback.before(tuple);
 
         int maxFieldPos = cmpFactories.length / 2;
         for (int i = 0; i < maxFieldPos; i++) {
@@ -232,6 +163,7 @@
                 throw new IllegalArgumentException("The low key point has larger coordinates than the high key point.");
             }
         }
+
         try {
             ICachedPage leafNode = findLeaf(ctx);
 
@@ -323,9 +255,9 @@
                 ctx.pathList.add(pageId, pageLsn, -1);
 
                 if (!isLeaf) {
-                    // findBestChild must be called *before* getBestChildPageId
-                    boolean enlarementIsNeeded = ctx.interiorFrame.findBestChild(ctx.getTuple(), ctx.cmp);
-                    int childPageId = ctx.interiorFrame.getBestChildPageId();
+                    // findBestChild must be called *before* checkIfEnlarementIsNeeded
+                    int childPageId = ctx.interiorFrame.findBestChild(ctx.getTuple(), ctx.cmp);
+                    boolean enlarementIsNeeded = ctx.interiorFrame.checkIfEnlarementIsNeeded(ctx.getTuple(), ctx.cmp);
 
                     if (enlarementIsNeeded) {
                         if (!writeLatched) {
@@ -404,6 +336,7 @@
                     if (!isLeaf) {
                         ctx.interiorFrame.insert(tuple, -1);
                     } else {
+                        ctx.modificationCallback.found(null, tuple);
                         ctx.leafFrame.insert(tuple, -1);
                     }
                     succeeded = true;
@@ -428,6 +361,7 @@
                         ctx.interiorFrame.insert(tuple, -1);
                     } else {
                         ctx.leafFrame.compact();
+                        ctx.modificationCallback.found(null, tuple);
                         ctx.leafFrame.insert(tuple, -1);
                     }
                     succeeded = true;
@@ -464,6 +398,7 @@
                         rightFrame.setPage(rightNode);
                         rightFrame.initBuffer((byte) 0);
                         rightFrame.setRightPage(ctx.interiorFrame.getRightPage());
+                        ctx.modificationCallback.found(null, tuple);
                         ctx.leafFrame.split(rightFrame, tuple, ctx.splitKey);
                         ctx.leafFrame.setRightPage(rightPageId);
                     }
@@ -795,12 +730,14 @@
     }
 
     private void deleteTuple(int tupleIndex, RTreeOpContext ctx) throws HyracksDataException {
+        ITupleReference beforeTuple = ctx.leafFrame.getBeforeTuple(ctx.getTuple(), tupleIndex, ctx.cmp);
+        ctx.modificationCallback.found(beforeTuple, ctx.getTuple());
         ctx.leafFrame.delete(tupleIndex, ctx.cmp);
         ctx.leafFrame.setPageLsn(incrementGlobalNsn());
     }
 
     private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, RTreeOpContext ctx)
-            throws HyracksDataException, TreeIndexException {
+            throws HyracksDataException, IndexException {
         ctx.reset();
         ctx.cursor = cursor;
 
@@ -810,90 +747,18 @@
         ctx.cursor.open(ctx.cursorInitialState, (SearchPredicate) searchPred);
     }
 
-    @Override
-    public ITreeIndexFrameFactory getInteriorFrameFactory() {
-        return interiorFrameFactory;
-    }
-
-    @Override
-    public ITreeIndexFrameFactory getLeafFrameFactory() {
-        return leafFrameFactory;
-    }
-
-    @Override
-    public IBinaryComparatorFactory[] getComparatorFactories() {
-        return cmpFactories;
-    }
-
-    @Override
-    public IFreePageManager getFreePageManager() {
-        return freePageManager;
-    }
-
     private void update(ITupleReference tuple, RTreeOpContext ctx) {
         throw new UnsupportedOperationException("RTree Update not implemented.");
     }
 
-    public boolean isEmptyTree(IRTreeLeafFrame leafFrame) throws HyracksDataException {
-        ICachedPage rootNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rootPage), false);
-        rootNode.acquireReadLatch();
-        try {
-            leafFrame.setPage(rootNode);
-            if (leafFrame.getLevel() == 0 && leafFrame.getTupleCount() == 0) {
-                return true;
-            } else {
-                return false;
-            }
-        } finally {
-            rootNode.releaseReadLatch();
-            bufferCache.unpin(rootNode);
-        }
-    }
-
-    public final class BulkLoadContext implements IIndexBulkLoadContext {
-
-        public ITreeIndexAccessor indexAccessor;
-
-        public BulkLoadContext(float fillFactor, IRTreeFrame leafFrame, IRTreeFrame interiorFrame,
-                ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException {
-            indexAccessor = createAccessor();
-        }
-    }
-
-    @Override
-    public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws HyracksDataException {
-        IRTreeLeafFrame leafFrame = (IRTreeLeafFrame) leafFrameFactory.createFrame();
-        if (!isEmptyTree(leafFrame)) {
-            throw new HyracksDataException("Trying to Bulk-load a non-empty RTree.");
-        }
-
-        BulkLoadContext ctx = new BulkLoadContext(fillFactor, (IRTreeFrame) leafFrameFactory.createFrame(),
-                (IRTreeFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
-                        .createFrame());
-        return ctx;
-    }
-
-    @Override
-    public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException {
-        try {
-            ((BulkLoadContext) ictx).indexAccessor.insert(tuple);
-        } catch (Exception e) {
-            throw new HyracksDataException("BulkLoad Error", e);
-        }
-    }
-
-    @Override
-    public void endBulkLoad(IIndexBulkLoadContext ictx) throws HyracksDataException {
-    }
-
     private void diskOrderScan(ITreeIndexCursor icursor, RTreeOpContext ctx) throws HyracksDataException {
-        TreeDiskOrderScanCursor cursor = (TreeDiskOrderScanCursor) icursor;
+        TreeIndexDiskOrderScanCursor cursor = (TreeIndexDiskOrderScanCursor) icursor;
         ctx.reset();
 
         MultiComparator cmp = MultiComparator.create(cmpFactories);
         SearchPredicate searchPred = new SearchPredicate(null, cmp);
 
-        int currentPageId = rootPage + 1;
+        int currentPageId = rootPage;
         int maxPageId = freePageManager.getMaxPage(ctx.metaFrame);
 
         ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
@@ -903,6 +768,7 @@
             cursor.setFileId(fileId);
             cursor.setCurrentPageId(currentPageId);
             cursor.setMaxPageId(maxPageId);
+            ctx.cursorInitialState.setOriginialKeyComparator(ctx.cmp);
             ctx.cursorInitialState.setPage(page);
             cursor.open(ctx.cursorInitialState, searchPred);
         } catch (Exception e) {
@@ -913,49 +779,36 @@
     }
 
     @Override
-    public int getRootPageId() {
-        return rootPage;
-    }
-
-    @Override
-    public int getFieldCount() {
-        return fieldCount;
-    }
-
-    @Override
-    public IndexType getIndexType() {
-        return IndexType.RTREE;
-    }
-
-    @Override
-    public ITreeIndexAccessor createAccessor() {
-        return new RTreeAccessor(this);
+    public ITreeIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
+            ISearchOperationCallback searchCallback) {
+        return new RTreeAccessor(this, modificationCallback, searchCallback);
     }
 
     public class RTreeAccessor implements ITreeIndexAccessor {
         private RTree rtree;
         private RTreeOpContext ctx;
 
-        public RTreeAccessor(RTree rtree) {
+        public RTreeAccessor(RTree rtree, IModificationOperationCallback modificationCallback,
+                ISearchOperationCallback searchCallback) {
             this.rtree = rtree;
-            this.ctx = rtree.createOpContext();
+            this.ctx = rtree.createOpContext(modificationCallback);
         }
 
         @Override
         public void insert(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
-            ctx.reset(IndexOp.INSERT);
+            ctx.setOperation(IndexOperation.INSERT);
             rtree.insert(tuple, ctx);
         }
 
         @Override
         public void update(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
-            ctx.reset(IndexOp.UPDATE);
+            ctx.setOperation(IndexOperation.UPDATE);
             rtree.update(tuple, ctx);
         }
 
         @Override
         public void delete(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
-            ctx.reset(IndexOp.DELETE);
+            ctx.setOperation(IndexOperation.DELETE);
             rtree.delete(tuple, ctx);
         }
 
@@ -968,18 +821,18 @@
         @Override
         public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
                 IndexException {
-            ctx.reset(IndexOp.SEARCH);
+            ctx.setOperation(IndexOperation.SEARCH);
             rtree.search((ITreeIndexCursor) cursor, searchPred, ctx);
         }
 
         @Override
         public ITreeIndexCursor createDiskOrderScanCursor() {
-            return new TreeDiskOrderScanCursor(leafFrameFactory.createFrame());
+            return new TreeIndexDiskOrderScanCursor(leafFrameFactory.createFrame());
         }
 
         @Override
         public void diskOrderScan(ITreeIndexCursor cursor) throws HyracksDataException {
-            ctx.reset(IndexOp.DISKORDERSCAN);
+            ctx.setOperation(IndexOperation.DISKORDERSCAN);
             rtree.diskOrderScan(cursor, ctx);
         }
 
@@ -990,7 +843,140 @@
         @Override
         public void upsert(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
             throw new UnsupportedOperationException(
-                    "The RTree does not suypport the notion of keys, therefore upsert does not make sense.");
+                    "The RTree does not support the notion of keys, therefore upsert does not make sense.");
         }
     }
+
+    @Override
+    public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint)
+            throws TreeIndexException {
+        // TODO: verifyInput currently does nothing.
+        try {
+            return new RTreeBulkLoader(fillFactor);
+        } catch (HyracksDataException e) {
+            throw new TreeIndexException(e);
+        }
+    }
+
+    public class RTreeBulkLoader extends AbstractTreeIndex.AbstractTreeIndexBulkLoader {
+        ITreeIndexFrame lowerFrame, prevInteriorFrame;
+        RTreeTypeAwareTupleWriter tupleWriter = ((RTreeTypeAwareTupleWriter) interiorFrame.getTupleWriter());
+        ITreeIndexTupleReference mbrTuple = interiorFrame.createTupleReference();
+        ByteBuffer mbr;
+
+        public RTreeBulkLoader(float fillFactor) throws TreeIndexException, HyracksDataException {
+            super(fillFactor);
+            prevInteriorFrame = interiorFrameFactory.createFrame();
+        }
+
+        @Override
+        public void add(ITupleReference tuple) throws HyracksDataException {
+            try {
+                NodeFrontier leafFrontier = nodeFrontiers.get(0);
+
+                int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
+                int spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+
+                // try to free space by compression
+                if (spaceUsed + spaceNeeded > leafMaxBytes) {
+                    leafFrame.compress();
+                    spaceUsed = leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace();
+                }
+
+                if (spaceUsed + spaceNeeded > leafMaxBytes) {
+                    propagateBulk(1, false);
+
+                    leafFrontier.pageId = freePageManager.getFreePage(metaFrame);
+
+                    leafFrontier.page.releaseWriteLatch();
+                    bufferCache.unpin(leafFrontier.page);
+
+                    leafFrontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId),
+                            true);
+                    leafFrontier.page.acquireWriteLatch();
+                    leafFrame.setPage(leafFrontier.page);
+                    leafFrame.initBuffer((byte) 0);
+                }
+
+                leafFrame.setPage(leafFrontier.page);
+                leafFrame.insert(tuple, -1);
+            } catch (HyracksDataException e) {
+                handleException();
+                throw e;
+            } catch (RuntimeException e) {
+                handleException();
+                throw e;
+            }
+
+        }
+
+        public void end() throws HyracksDataException {
+            propagateBulk(1, true);
+
+            super.end();
+        }
+
+        protected void propagateBulk(int level, boolean toRoot) throws HyracksDataException {
+            boolean propagated = false;
+
+            if (level == 1)
+                lowerFrame = leafFrame;
+
+            if (lowerFrame.getTupleCount() == 0)
+                return;
+
+            if (level >= nodeFrontiers.size())
+                addLevel();
+
+            ((RTreeNSMFrame) lowerFrame).adjustMBR();
+
+            if (mbr == null) {
+                int bytesRequired = tupleWriter.bytesRequired(((RTreeNSMFrame) lowerFrame).getTuples()[0], 0,
+                        cmp.getKeyFieldCount())
+                        + ((RTreeNSMInteriorFrame) interiorFrame).getChildPointerSize();
+                mbr = ByteBuffer.allocate(bytesRequired);
+            }
+            tupleWriter.writeTupleFields(((RTreeNSMFrame) lowerFrame).getTuples(), 0, mbr, 0);
+            mbrTuple.resetByTupleOffset(mbr, 0);
+
+            NodeFrontier frontier = nodeFrontiers.get(level);
+            interiorFrame.setPage(frontier.page);
+
+            interiorFrame.insert(mbrTuple, -1);
+
+            interiorFrame.getBuffer().putInt(
+                    interiorFrame.getTupleOffset(interiorFrame.getTupleCount() - 1) + mbrTuple.getTupleSize(),
+                    nodeFrontiers.get(level - 1).pageId);
+
+            if (interiorFrame.hasSpaceInsert(mbrTuple) != FrameOpSpaceStatus.SUFFICIENT_CONTIGUOUS_SPACE && !toRoot) {
+                lowerFrame = prevInteriorFrame;
+                lowerFrame.setPage(frontier.page);
+
+                propagateBulk(level + 1, toRoot);
+                propagated = true;
+
+                frontier.page.releaseWriteLatch();
+                bufferCache.unpin(frontier.page);
+                frontier.pageId = freePageManager.getFreePage(metaFrame);
+
+                frontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, frontier.pageId), true);
+                frontier.page.acquireWriteLatch();
+                interiorFrame.setPage(frontier.page);
+                interiorFrame.initBuffer((byte) level);
+            }
+
+            if (toRoot && !propagated && level < nodeFrontiers.size() - 1) {
+                lowerFrame = prevInteriorFrame;
+                lowerFrame.setPage(frontier.page);
+                propagateBulk(level + 1, true);
+            }
+
+            leafFrame.setPage(nodeFrontiers.get(0).page);
+        }
+    }
+
+    @Override
+    public void validate() throws HyracksDataException {
+        throw new UnsupportedOperationException("Validation not implemented for R-Trees.");
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeCursorInitialState.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeCursorInitialState.java
index ac1eb7d..8a7ea8d 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeCursorInitialState.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeCursorInitialState.java
@@ -16,36 +16,59 @@
 package edu.uci.ics.hyracks.storage.am.rtree.impls;
 
 import edu.uci.ics.hyracks.storage.am.common.api.ICursorInitialState;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 
 public class RTreeCursorInitialState implements ICursorInitialState {
 
-	private PathList pathList;
-	private int rootPage;
-	private ICachedPage page; // for disk order scan
+    private PathList pathList;
+    private int rootPage;
+    private ICachedPage page; // for disk order scan
+    private MultiComparator originalKeyCmp;
 
-	public RTreeCursorInitialState(PathList pathList, int rootPage) {
-		this.pathList = pathList;
-		this.rootPage = rootPage;
-	}
+    public RTreeCursorInitialState(PathList pathList, int rootPage) {
+        this.pathList = pathList;
+        this.rootPage = rootPage;
+    }
 
-	public PathList getPathList() {
-		return pathList;
-	}
+    public PathList getPathList() {
+        return pathList;
+    }
 
-	public int getRootPage() {
-		return rootPage;
-	}
+    public int getRootPage() {
+        return rootPage;
+    }
 
-	public void setRootPage(int rootPage) {
-		this.rootPage = rootPage;
-	}
+    public void setRootPage(int rootPage) {
+        this.rootPage = rootPage;
+    }
 
-	public ICachedPage getPage() {
-		return page;
-	}
+    public ICachedPage getPage() {
+        return page;
+    }
 
-	public void setPage(ICachedPage page) {
-		this.page = page;
-	}
+    public void setPage(ICachedPage page) {
+        this.page = page;
+    }
+
+    @Override
+    public MultiComparator getOriginalKeyComparator() {
+        return originalKeyCmp;
+    }
+
+    @Override
+    public void setOriginialKeyComparator(MultiComparator originalCmp) {
+        this.originalKeyCmp = originalCmp;
+    }
+
+    @Override
+    public ISearchOperationCallback getSearchOperationCallback() {
+        return null;
+    }
+
+    @Override
+    public void setSearchOperationCallback(ISearchOperationCallback searchCallback) {
+        // Do nothing
+    }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
index 6683444..219ab30 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
@@ -19,21 +19,22 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 
-public class RTreeOpContext implements IIndexOpContext {
+public class RTreeOpContext implements IIndexOperationContext {
     private static final int INITIAL_TRAVERSE_LIST_SIZE = 100;
     public final MultiComparator cmp;
     public final IRTreeInteriorFrame interiorFrame;
     public final IRTreeLeafFrame leafFrame;
-    public IndexOp op;
+    public IndexOperation op;
     public ITreeIndexCursor cursor;
     public RTreeCursorInitialState cursorInitialState;
     public ITreeIndexMetaDataFrame metaFrame;
@@ -47,12 +48,22 @@
     public ArrayList<ICachedPage> NSNUpdates;
     public ArrayList<ICachedPage> LSNUpdates;
 
+    public final IModificationOperationCallback modificationCallback;
+
     public RTreeOpContext(IRTreeLeafFrame leafFrame, IRTreeInteriorFrame interiorFrame,
-            ITreeIndexMetaDataFrame metaFrame, IBinaryComparatorFactory[] cmpFactories, int treeHeightHint) {
-        this.cmp = MultiComparator.create(cmpFactories);
+            ITreeIndexMetaDataFrame metaFrame, IBinaryComparatorFactory[] cmpFactories, int treeHeightHint,
+            IModificationOperationCallback modificationCallback) {
+        
+        if (cmpFactories[0] != null) { 
+            this.cmp = MultiComparator.create(cmpFactories);
+        } else {
+            this.cmp = null;
+        }
+        
         this.interiorFrame = interiorFrame;
         this.leafFrame = leafFrame;
         this.metaFrame = metaFrame;
+        this.modificationCallback = modificationCallback;
         pathList = new PathList(treeHeightHint, treeHeightHint);
         NSNUpdates = new ArrayList<ICachedPage>();
         LSNUpdates = new ArrayList<ICachedPage>();
@@ -78,11 +89,11 @@
     }
 
     @Override
-    public void reset(IndexOp newOp) {
+    public void setOperation(IndexOperation newOp) {
         if (op != null && newOp == op) {
             return;
         }
-        if (op != IndexOp.SEARCH && op != IndexOp.DISKORDERSCAN) {
+        if (op != IndexOperation.SEARCH && op != IndexOperation.DISKORDERSCAN) {
             if (splitKey == null) {
                 splitKey = new RTreeSplitKey(interiorFrame.getTupleWriter().createTupleReference(), interiorFrame
                         .getTupleWriter().createTupleReference());
@@ -96,4 +107,9 @@
         }
         this.op = newOp;
     }
+
+    @Override
+    public IndexOperation getOperation() {
+        return op;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
index ee7ec5f..6b5b1b5 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
@@ -33,20 +33,20 @@
     private int fileId = -1;
     private ICachedPage page = null;
     private IRTreeInteriorFrame interiorFrame = null;
-    private IRTreeLeafFrame leafFrame = null;
+    protected IRTreeLeafFrame leafFrame = null;
     private IBufferCache bufferCache = null;
 
     private SearchPredicate pred;
     private PathList pathList;
     private int rootPage;
-    private ITupleReference searchKey;
+    protected ITupleReference searchKey;
 
     private int tupleIndex = 0;
     private int tupleIndexInc = 0;
     private int currentTupleIndex = 0;
     private int pageId = -1;
 
-    private MultiComparator cmp;
+    protected MultiComparator cmp;
 
     private ITreeIndexTupleReference frameTuple;
     private boolean readLatched = false;
@@ -70,6 +70,7 @@
         pathList = null;
     }
 
+    @Override
     public ITupleReference getTuple() {
         return frameTuple;
     }
@@ -87,7 +88,7 @@
         return page;
     }
 
-    private boolean fetchNextLeafPage() throws HyracksDataException {
+    protected boolean fetchNextLeafPage() throws HyracksDataException {
         boolean succeeded = false;
         if (readLatched) {
             page.releaseReadLatch();
@@ -117,15 +118,18 @@
                 }
 
                 if (!isLeaf) {
+                    // We do DFS so that we get the tuples ordered (for disk
+                    // RTrees only) in the case we we are using total order
+                    // (such as Hilbert order)
                     if (searchKey != null) {
-                        for (int i = 0; i < interiorFrame.getTupleCount(); i++) {
+                        for (int i = interiorFrame.getTupleCount() - 1; i >= 0; i--) {
                             int childPageId = interiorFrame.getChildPageIdIfIntersect(searchKey, i, cmp);
                             if (childPageId != -1) {
                                 pathList.add(childPageId, pageLsn, -1);
                             }
                         }
                     } else {
-                        for (int i = 0; i < interiorFrame.getTupleCount(); i++) {
+                        for (int i = interiorFrame.getTupleCount() - 1; i >= 0; i--) {
                             int childPageId = interiorFrame.getChildPageId(i);
                             pathList.add(childPageId, pageLsn, -1);
                         }
@@ -230,12 +234,8 @@
     }
 
     @Override
-    public void reset() {
-        try {
-            close();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
+    public void reset() throws HyracksDataException {
+        close();
     }
 
     @Override
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/Rectangle.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/Rectangle.java
index cb9b160..d0f4c71 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/Rectangle.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/Rectangle.java
@@ -19,94 +19,120 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;
 
 public class Rectangle {
-	private int dim;
-	private double[] low;
-	private double[] high;
+    private int dim;
+    private double[] low;
+    private double[] high;
 
-	public Rectangle(int dim) {
-		this.dim = dim;
-		low = new double[this.dim];
-		high = new double[this.dim];
-	}
+    public Rectangle(int dim) {
+        this.dim = dim;
+        low = new double[this.dim];
+        high = new double[this.dim];
+    }
 
-	public int getDim() {
-		return dim;
-	}
+    public int getDim() {
+        return dim;
+    }
 
-	public double getLow(int i) {
-		return low[i];
-	}
+    public double getLow(int i) {
+        return low[i];
+    }
 
-	public double getHigh(int i) {
-		return high[i];
-	}
+    public double getHigh(int i) {
+        return high[i];
+    }
 
-	public void setLow(int i, double value) {
-		low[i] = value;
-	}
+    public void setLow(int i, double value) {
+        low[i] = value;
+    }
 
-	public void setHigh(int i, double value) {
-		high[i] = value;
-	}
+    public void setHigh(int i, double value) {
+        high[i] = value;
+    }
 
-	public void set(ITupleReference tuple, IPrimitiveValueProvider[] valueProviders) {
-		for (int i = 0; i < getDim(); i++) {
-			int j = i + getDim();
-			setLow(i, valueProviders[i].getValue(
-					tuple.getFieldData(i), tuple.getFieldStart(i)));
-			setHigh(i, valueProviders[j].getValue(
-					tuple.getFieldData(j), tuple.getFieldStart(j)));
-		}
-	}
+    public void set(ITupleReference tuple, IPrimitiveValueProvider[] valueProviders) {
+        for (int i = 0; i < getDim(); i++) {
+            int j = i + getDim();
+            setLow(i, valueProviders[i].getValue(tuple.getFieldData(i), tuple.getFieldStart(i)));
+            setHigh(i, valueProviders[j].getValue(tuple.getFieldData(j), tuple.getFieldStart(j)));
+        }
+    }
 
-	public void enlarge(ITupleReference tupleToBeInserted, IPrimitiveValueProvider[] valueProviders) {
-		for (int i = 0; i < getDim(); i++) {
-			int j = getDim() + i;
-			double low = valueProviders[i].getValue(
-					tupleToBeInserted.getFieldData(i),
-					tupleToBeInserted.getFieldStart(i));
-			if (getLow(i) > low) {
-				setLow(i, low);
-			}
-			double high = valueProviders[j].getValue(
-					tupleToBeInserted.getFieldData(j),
-					tupleToBeInserted.getFieldStart(j));
-			if (getHigh(i) < high) {
-				setHigh(i, high);
-			}
-		}
-	}
+    public void enlarge(ITupleReference tupleToBeInserted, IPrimitiveValueProvider[] valueProviders) {
+        for (int i = 0; i < getDim(); i++) {
+            int j = getDim() + i;
+            double low = valueProviders[i].getValue(tupleToBeInserted.getFieldData(i),
+                    tupleToBeInserted.getFieldStart(i));
+            if (getLow(i) > low) {
+                setLow(i, low);
+            }
+            double high = valueProviders[j].getValue(tupleToBeInserted.getFieldData(j),
+                    tupleToBeInserted.getFieldStart(j));
+            if (getHigh(i) < high) {
+                setHigh(i, high);
+            }
+        }
+    }
 
-	public double margin() {
-		double margin = 0.0;
-		double mul = Math.pow(2, (double) getDim() - 1.0);
-		for (int i = 0; i < getDim(); i++) {
-			margin += (getHigh(i) - getLow(i)) * mul;
-		}
-		return margin;
-	}
+    public double enlargedArea(ITupleReference tupleToBeInserted, IPrimitiveValueProvider[] valueProviders) {
+        double areaBeforeEnlarge = area();
+        double areaAfterEnlarge = 1.0;
 
-	public double overlappedArea(Rectangle rec) {
-		double area = 1.0;
-		double f1, f2;
+        for (int i = 0; i < getDim(); i++) {
+            int j = getDim() + i;
 
-		for (int i = 0; i < getDim(); i++) {
-			if (getLow(i) > rec.getHigh(i) || getHigh(i) < rec.getLow(i)) {
-				return 0.0;
-			}
+            double low = valueProviders[i].getValue(tupleToBeInserted.getFieldData(i),
+                    tupleToBeInserted.getFieldStart(i));
+            double lowAfterEnlargement;
+            if (getLow(i) > low) {
+                lowAfterEnlargement = low;
+            } else {
+                lowAfterEnlargement = getLow(i);
+            }
 
-			f1 = Math.max(getLow(i), rec.getLow(i));
-			f2 = Math.min(getHigh(i), rec.getHigh(i));
-			area *= f2 - f1;
-		}
-		return area;
-	}
+            double high = valueProviders[j].getValue(tupleToBeInserted.getFieldData(j),
+                    tupleToBeInserted.getFieldStart(j));
+            double highAfterEnlargement;
+            if (getHigh(i) < high) {
+                highAfterEnlargement = high;
+            } else {
+                highAfterEnlargement = getHigh(i);
+            }
 
-	public double area() {
-		double area = 1.0;
-		for (int i = 0; i < getDim(); i++) {
-			area *= getHigh(i) - getLow(i);
-		}
-		return area;
-	}
+            areaAfterEnlarge *= highAfterEnlargement - lowAfterEnlargement;
+        }
+        return areaAfterEnlarge - areaBeforeEnlarge;
+    }
+
+    public double margin() {
+        double margin = 0.0;
+        double mul = Math.pow(2, (double) getDim() - 1.0);
+        for (int i = 0; i < getDim(); i++) {
+            margin += (getHigh(i) - getLow(i)) * mul;
+        }
+        return margin;
+    }
+
+    public double overlappedArea(Rectangle rec) {
+        double area = 1.0;
+        double f1, f2;
+
+        for (int i = 0; i < getDim(); i++) {
+            if (getLow(i) > rec.getHigh(i) || getHigh(i) < rec.getLow(i)) {
+                return 0.0;
+            }
+
+            f1 = Math.max(getLow(i), rec.getLow(i));
+            f2 = Math.min(getHigh(i), rec.getHigh(i));
+            area *= f2 - f1;
+        }
+        return area;
+    }
+
+    public double area() {
+        double area = 1.0;
+        for (int i = 0; i < getDim(); i++) {
+            area *= getHigh(i) - getLow(i);
+        }
+        return area;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
new file mode 100644
index 0000000..7fce7e0
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
@@ -0,0 +1,180 @@
+package edu.uci.ics.hyracks.storage.am.rtree.linearize;

+

+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparator;

+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;

+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;

+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;

+import edu.uci.ics.hyracks.storage.am.common.ophelpers.DoubleArrayList;

+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IntArrayList;

+import edu.uci.ics.hyracks.storage.am.rtree.impls.DoublePrimitiveValueProviderFactory;

+

+/*

+ * This compares two points based on the hilbert curve. Currently, it only supports

+ * doubles (this can be changed by changing all doubles to ints as there are no

+ * number generics in Java) in the two-dimensional space. For more dimensions, the

+ * state machine has to be automatically generated. The idea of the fractal generation

+ * of the curve is described e.g. in http://dl.acm.org/ft_gateway.cfm?id=383528&type=pdf

+ * 

+ * Unlike the described approach, this comparator does not compute the hilbert value at 

+ * any point. Instead, it only evaluates how the two inputs compare to each other. This

+ * is done by starting at the lowest hilbert resolution and zooming in on the fractal until

+ * the two points are in different quadrants.

+ * 

+ * As a performance optimization, the state of the state machine is saved in a stack and 

+ * maintained over comparisons. The idea behind this is that comparisons are usually in a

+ * similar area (e.g. geo coordinates). Zooming in from [-MAX_VALUE, MAX_VALUE] would take

+ * ~300 steps every time. Instead, the comparator start from the previous state and zooms out

+ * if necessary

+ */

+

+public class HilbertDoubleComparator implements ILinearizeComparator {

+    private final int dim; // dimension

+    private final HilbertState[] states;

+

+    private double[] bounds;

+    private double stepsize;

+    private int state;

+    private IntArrayList stateStack = new IntArrayList(1000, 200);

+    private DoubleArrayList boundsStack = new DoubleArrayList(2000, 400);

+

+    private IPrimitiveValueProvider valueProvider = DoublePrimitiveValueProviderFactory.INSTANCE

+            .createPrimitiveValueProvider();

+

+    private double[] a;

+    private double[] b;

+

+    private class HilbertState {

+        public final int[] nextState;

+        public final int[] position;

+

+        public HilbertState(int[] nextState, int[] order) {

+            this.nextState = nextState;

+            this.position = order;

+        }

+    }

+

+    public HilbertDoubleComparator(int dimension) {

+        if (dimension != 2)

+            throw new IllegalArgumentException();

+        dim = dimension;

+        a = new double[dim];

+        b = new double[dim];

+

+        states = new HilbertState[] { new HilbertState(new int[] { 3, 0, 1, 0 }, new int[] { 0, 1, 3, 2 }),

+                new HilbertState(new int[] { 1, 1, 0, 2 }, new int[] { 2, 1, 3, 0 }),

+                new HilbertState(new int[] { 2, 3, 2, 1 }, new int[] { 2, 3, 1, 0 }),

+                new HilbertState(new int[] { 0, 2, 3, 3 }, new int[] { 0, 3, 1, 2 }) };

+

+        resetStateMachine();

+    }

+

+    private void resetStateMachine() {

+        state = 0;

+        stateStack.clear();

+        stepsize = Double.MAX_VALUE / 2;

+        bounds = new double[dim];

+        boundsStack.clear();

+    }

+

+    public int compare() {

+        boolean equal = true;

+        for (int i = 0; i < dim; i++) {

+            if (a[i] != b[i])

+                equal = false;

+        }

+        if (equal)

+            return 0;

+

+        // We keep the state of the state machine after a comparison. In most

+        // cases,

+        // the needed zoom factor is close to the old one. In this step, we

+        // check if we have

+        // to zoom out

+        while (true) {

+            if (stateStack.size() <= dim) {

+                resetStateMachine();

+                break;

+            }

+            boolean zoomOut = false;

+            for (int i = 0; i < dim; i++) {

+                if (Math.min(a[i], b[i]) <= bounds[i] - 2 * stepsize

+                        || Math.max(a[i], b[i]) >= bounds[i] + 2 * stepsize) {

+                    zoomOut = true;

+                    break;

+                }

+            }

+            state = stateStack.getLast();

+            stateStack.removeLast();

+            for (int j = dim - 1; j >= 0; j--) {

+                bounds[j] = boundsStack.getLast();

+                boundsStack.removeLast();

+            }

+            stepsize *= 2;

+            if (!zoomOut) {

+                state = stateStack.getLast();

+                stateStack.removeLast();

+                for (int j = dim - 1; j >= 0; j--) {

+                    bounds[j] = boundsStack.getLast();

+                    boundsStack.removeLast();

+                }

+                stepsize *= 2;

+                break;

+            }

+        }

+

+        while (true) {

+            stateStack.add(state);

+            for (int j = 0; j < dim; j++) {

+                boundsStack.add(bounds[j]);

+            }

+

+            // Find the quadrant in which A and B are

+            int quadrantA = 0, quadrantB = 0;

+            for (int i = dim - 1; i >= 0; i--) {

+                if (a[i] >= bounds[i])

+                    quadrantA ^= (1 << (dim - i - 1));

+                if (b[i] >= bounds[i])

+                    quadrantB ^= (1 << (dim - i - 1));

+

+                if (a[i] >= bounds[i]) {

+                    bounds[i] += stepsize;

+                } else {

+                    bounds[i] -= stepsize;

+                }

+            }

+

+            stepsize /= 2;

+            if (stepsize <= 2 * DoublePointable.getEpsilon())

+                return 0;

+            // avoid infinite loop due to machine epsilon problems

+

+            if (quadrantA != quadrantB) {

+                // find the position of A and B's quadrants

+                int posA = states[state].position[quadrantA];

+                int posB = states[state].position[quadrantB];

+

+                if (posA < posB)

+                    return -1;

+                else

+                    return 1;

+            }

+

+            state = states[state].nextState[quadrantA];

+        }

+    }

+

+    @Override

+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {

+        for (int i = 0; i < dim; i++) {

+            a[i] = DoubleSerializerDeserializer.getDouble(b1, s1 + (i * 8));

+            b[i] = DoubleSerializerDeserializer.getDouble(b2, s2 + (i * 8));

+        }

+

+        return compare();

+    }

+

+    @Override

+    public int getDimensions() {

+        return dim;

+    }

+}

diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/HilbertDoubleComparatorFactory.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/HilbertDoubleComparatorFactory.java
new file mode 100644
index 0000000..e06dba8
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/HilbertDoubleComparatorFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 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.rtree.linearize;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+
+public class HilbertDoubleComparatorFactory implements ILinearizeComparatorFactory {
+    private static final long serialVersionUID = 1L;
+    
+    private int dim;
+
+    public static HilbertDoubleComparatorFactory get(int dim) {
+        return new HilbertDoubleComparatorFactory(dim);
+    }
+    
+    public HilbertDoubleComparatorFactory(int dim) {
+    	this.dim = dim;
+    }
+
+    @Override
+    public ILinearizeComparator createBinaryComparator() {
+        return new HilbertDoubleComparator(dim);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
new file mode 100644
index 0000000..ee47761
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
@@ -0,0 +1,136 @@
+package edu.uci.ics.hyracks.storage.am.rtree.linearize;

+

+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparator;

+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;

+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;

+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProvider;

+import edu.uci.ics.hyracks.storage.am.common.ophelpers.DoubleArrayList;

+import edu.uci.ics.hyracks.storage.am.rtree.impls.DoublePrimitiveValueProviderFactory;

+

+/*

+ * This compares two points based on the z curve. For doubles, we cannot use

+ * the simple bit magic approach. There may, however, be a better approach than this.

+ */

+

+public class ZCurveDoubleComparator implements ILinearizeComparator {

+    private final int dim; // dimension

+

+    private double[] bounds;

+    private double stepsize;

+    private DoubleArrayList boundsStack = new DoubleArrayList(2000, 400);

+

+    private IPrimitiveValueProvider valueProvider = DoublePrimitiveValueProviderFactory.INSTANCE

+            .createPrimitiveValueProvider();

+

+    private double[] a;

+    private double[] b;

+

+    public ZCurveDoubleComparator(int dimension) {

+        dim = dimension;

+        a = new double[dim];

+        b = new double[dim];

+

+        resetStateMachine();

+    }

+

+    private void resetStateMachine() {

+        stepsize = Double.MAX_VALUE / 2;

+        bounds = new double[dim];

+        boundsStack.clear();

+    }

+

+    public int compare() {

+        boolean equal = true;

+        for (int i = 0; i < dim; i++) {

+            if (a[i] != b[i])

+                equal = false;

+        }

+        if (equal)

+            return 0;

+

+        // We keep the state of the state machine after a comparison. In most

+        // cases,

+        // the needed zoom factor is close to the old one. In this step, we

+        // check if we have

+        // to zoom out

+        while (true) {

+            if (boundsStack.size() <= dim) {

+                resetStateMachine();

+                break;

+            }

+            boolean zoomOut = false;

+            for (int i = 0; i < dim; i++) {

+                if (Math.min(a[i], b[i]) <= bounds[i] - 2 * stepsize

+                        || Math.max(a[i], b[i]) >= bounds[i] + 2 * stepsize) {

+                    zoomOut = true;

+                    break;

+                }

+            }

+

+            for (int j = dim - 1; j >= 0; j--) {

+                bounds[j] = boundsStack.getLast();

+                boundsStack.removeLast();

+            }

+            stepsize *= 2;

+            if (!zoomOut) {

+                for (int j = dim - 1; j >= 0; j--) {

+                    bounds[j] = boundsStack.getLast();

+                    boundsStack.removeLast();

+                }

+                stepsize *= 2;

+                break;

+            }

+        }

+

+        while (true) {

+            for (int j = 0; j < dim; j++) {

+                boundsStack.add(bounds[j]);

+            }

+

+            // Find the quadrant in which A and B are

+            int quadrantA = 0, quadrantB = 0;

+            for (int i = dim - 1; i >= 0; i--) {

+                if (a[i] >= bounds[i])

+                    quadrantA ^= (1 << (dim - i - 1));

+                if (b[i] >= bounds[i])

+                    quadrantB ^= (1 << (dim - i - 1));

+

+                if (a[i] >= bounds[i]) {

+                    bounds[i] += stepsize;

+                } else {

+                    bounds[i] -= stepsize;

+                }

+            }

+

+            stepsize /= 2;

+            if (stepsize <= 2 * DoublePointable.getEpsilon())

+                return 0;

+            // avoid infinite loop due to machine epsilon problems

+

+            if (quadrantA != quadrantB) {

+                // find the position of A and B's quadrants

+                if (quadrantA < quadrantB)

+                    return -1;

+                else if (quadrantA > quadrantB)

+                    return 1;

+                else

+                    return 0;

+            }

+        }

+    }

+

+    @Override

+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {

+        for (int i = 0; i < dim; i++) {

+            a[i] = DoubleSerializerDeserializer.getDouble(b1, s1 + (i * 8));

+            b[i] = DoubleSerializerDeserializer.getDouble(b2, s2 + (i * 8));

+        }

+

+        return compare();

+    }

+

+    @Override

+    public int getDimensions() {

+        return dim;

+    }

+}

diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparatorFactory.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparatorFactory.java
new file mode 100644
index 0000000..f1b5806
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparatorFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 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.rtree.linearize;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+
+public class ZCurveDoubleComparatorFactory implements ILinearizeComparatorFactory {
+    private static final long serialVersionUID = 1L;
+    
+    private int dim;
+
+    public static ZCurveDoubleComparatorFactory get(int dim) {
+        return new ZCurveDoubleComparatorFactory(dim);
+    }
+    
+    public ZCurveDoubleComparatorFactory(int dim) {
+    	this.dim = dim;
+    }
+
+    @Override
+    public ILinearizeComparator createBinaryComparator() {
+        return new ZCurveDoubleComparator(dim);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
new file mode 100644
index 0000000..1f26f41
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
@@ -0,0 +1,129 @@
+package edu.uci.ics.hyracks.storage.am.rtree.linearize;

+

+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparator;

+import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;

+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;

+import edu.uci.ics.hyracks.storage.am.common.ophelpers.DoubleArrayList;

+

+/*

+ * This compares two points based on the z curve. For doubles, we cannot use

+ * the simple bit magic approach. There may, however, be a better approach than this.

+ */

+

+public class ZCurveIntComparator implements ILinearizeComparator {

+    private final int dim; // dimension

+

+    private double[] bounds;

+    private double stepsize;

+    private DoubleArrayList boundsStack = new DoubleArrayList(2000, 400);

+

+    private int[] a;

+    private int[] b;

+

+    public ZCurveIntComparator(int dimension) {

+        dim = dimension;

+        a = new int[dim];

+        b = new int[dim];

+

+        resetStateMachine();

+    }

+

+    private void resetStateMachine() {

+        stepsize = Integer.MAX_VALUE / 2;

+        bounds = new double[dim];

+        boundsStack.clear();

+    }

+

+    public int compare() {

+        boolean equal = true;

+        for (int i = 0; i < dim; i++) {

+            if (a[i] != b[i])

+                equal = false;

+        }

+        if (equal)

+            return 0;

+

+        // We keep the state of the state machine after a comparison. In most cases,

+        // the needed zoom factor is close to the old one. In this step, we check if we have

+        // to zoom out

+        while (true) {

+            if (boundsStack.size() <= dim) {

+                resetStateMachine();

+                break;

+            }

+            boolean zoomOut = false;

+            for (int i = 0; i < dim; i++) {

+                if (Math.min(a[i], b[i]) <= bounds[i] - 2 * stepsize

+                        || Math.max(a[i], b[i]) >= bounds[i] + 2 * stepsize) {

+                    zoomOut = true;

+                    break;

+                }

+            }

+

+            for (int j = dim - 1; j >= 0; j--) {

+                bounds[j] = boundsStack.getLast();

+                boundsStack.removeLast();

+            }

+            stepsize *= 2;

+            if (!zoomOut) {

+                for (int j = dim - 1; j >= 0; j--) {

+                    bounds[j] = boundsStack.getLast();

+                    boundsStack.removeLast();

+                }

+                stepsize *= 2;

+                break;

+            }

+        }

+

+        while (true) {

+            for (int j = 0; j < dim; j++) {

+                boundsStack.add(bounds[j]);

+            }

+

+            // Find the quadrant in which A and B are

+            int quadrantA = 0, quadrantB = 0;

+            for (int i = dim - 1; i >= 0; i--) {

+                if (a[i] >= bounds[i])

+                    quadrantA ^= (1 << (dim - i - 1));

+                if (b[i] >= bounds[i])

+                    quadrantB ^= (1 << (dim - i - 1));

+

+                if (a[i] >= bounds[i]) {

+                    bounds[i] += stepsize;

+                } else {

+                    bounds[i] -= stepsize;

+                }

+            }

+

+            stepsize /= 2;

+            if (stepsize <= 2 * DoublePointable.getEpsilon())

+                return 0;

+            // avoid infinite loop due to machine epsilon problems

+

+            if (quadrantA != quadrantB) {

+                // find the position of A and B's quadrants

+                if (quadrantA < quadrantB)

+                    return -1;

+                else if (quadrantA > quadrantB)

+                    return 1;

+                else

+                    return 0;

+            }

+        }

+    }

+

+    @Override

+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {

+        for (int i = 0; i < dim; i++) {

+            a[i] = IntegerSerializerDeserializer.getInt(b1, s1 + (i * 8));

+            b[i] = IntegerSerializerDeserializer.getInt(b2, s2 + (i * 8));

+        }

+

+        return compare();

+    }

+

+    @Override

+    public int getDimensions() {

+        return dim;

+    }

+}

diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveIntComparatorFactory.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveIntComparatorFactory.java
new file mode 100644
index 0000000..4a35a79
--- /dev/null
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/linearize/ZCurveIntComparatorFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 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.rtree.linearize;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+
+public class ZCurveIntComparatorFactory implements ILinearizeComparatorFactory {
+    private static final long serialVersionUID = 1L;
+    
+    private int dim;
+
+    public static ZCurveIntComparatorFactory get(int dim) {
+        return new ZCurveIntComparatorFactory(dim);
+    }
+    
+    public ZCurveIntComparatorFactory(int dim) {
+    	this.dim = dim;
+    }
+
+    @Override
+    public ILinearizeComparator createBinaryComparator() {
+        return new ZCurveIntComparator(dim);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java
index 6f5d36f..5889abb 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/util/RTreeUtils.java
@@ -18,6 +18,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 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.data.std.api.IPointableFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
@@ -30,24 +31,27 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
 import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class RTreeUtils {
-    public static RTree createRTree(IBufferCache bufferCache, ITypeTraits[] typeTraits,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, IBinaryComparatorFactory[] cmpFactories) {
+    public static RTree createRTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            ITypeTraits[] typeTraits, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            IBinaryComparatorFactory[] cmpFactories, RTreePolicyType rtreePolicyType, FileReference file) {
 
         RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
-                valueProviderFactories);
+                valueProviderFactories, rtreePolicyType);
         ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
-                valueProviderFactories);
+                valueProviderFactories, rtreePolicyType);
         ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
-        RTree rtree = new RTree(bufferCache, typeTraits.length, cmpFactories, freePageManager, interiorFrameFactory,
-                leafFrameFactory);
+        RTree rtree = new RTree(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory,
+                cmpFactories, typeTraits.length, file);
         return rtree;
     }
 
diff --git a/hyracks/hyracks-storage-common/pom.xml b/hyracks/hyracks-storage-common/pom.xml
index 3360097..25a6c25 100644
--- a/hyracks/hyracks-storage-common/pom.xml
+++ b/hyracks/hyracks-storage-common/pom.xml
@@ -1,8 +1,6 @@
 <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>hyracks-storage-common</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-storage-common</name>
 
   <parent>
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
index 562305e..5759218 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
@@ -19,9 +19,15 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 
 public interface IStorageManagerInterface extends Serializable {
     public IBufferCache getBufferCache(IHyracksTaskContext ctx);
 
     public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx);
+
+    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx);
+
+    public ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx);
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
index 7f17f38..03e46f5 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
@@ -137,19 +137,11 @@
         pinSanityCheck(dpid);
         CachedPage cPage = findPage(dpid, newPage);
         if (!newPage) {
-            if (!cPage.valid) {
-                /*
-                 * We got a buffer and we have pinned it. But its invalid. If its a new page, we just mark it as valid
-                 * and return. Or else, while we hold the page lock, we get a write latch on the data and start a read.
-                 */
-                cPage.acquireWriteLatch(false);
-                try {
-                    if (!cPage.valid) {
-                        read(cPage);
-                    }
+            // Resolve race of multiple threads trying to read the page from disk.
+            synchronized (cPage) {
+                if (!cPage.valid) {
+                    read(cPage);
                     cPage.valid = true;
-                } finally {
-                    cPage.releaseWriteLatch();
                 }
             }
         } else {
@@ -774,9 +766,11 @@
                 if (fInfo != null) {
                     // Mark the fInfo as deleted, 
                     // such that when its pages are reclaimed in openFile(),
-                    // the pages are not flushed to disk but only invalidates.
-                    ioManager.close(fInfo.getFileHandle());
-                    fInfo.markAsDeleted();
+                    // the pages are not flushed to disk but only invalidated.
+                    if (!fInfo.fileHasBeenDeleted()) {
+                        ioManager.close(fInfo.getFileHandle());
+                        fInfo.markAsDeleted();
+                    }
                 }
             }
         }
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/DebugBufferCache.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/DebugBufferCache.java
index d610c7e..13f7d52 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/DebugBufferCache.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/DebugBufferCache.java
@@ -30,16 +30,16 @@
 
     // Actual BufferCache functionality is delegated to this bufferCache.
     private final IBufferCache bufferCache;
-    private AtomicLong pinCount;
-    private AtomicLong unpinCount;
-    private AtomicLong readLatchCount;
-    private AtomicLong readUnlatchCount;
-    private AtomicLong writeLatchCount;
-    private AtomicLong writeUnlatchCount;
-    private AtomicLong createFileCount;
-    private AtomicLong deleteFileCount;
-    private AtomicLong openFileCount;
-    private AtomicLong closeFileCount;
+    private AtomicLong pinCount = new AtomicLong();
+    private AtomicLong unpinCount = new AtomicLong();
+    private AtomicLong readLatchCount = new AtomicLong();
+    private AtomicLong readUnlatchCount = new AtomicLong();
+    private AtomicLong writeLatchCount = new AtomicLong();
+    private AtomicLong writeUnlatchCount = new AtomicLong();
+    private AtomicLong createFileCount = new AtomicLong();
+    private AtomicLong deleteFileCount = new AtomicLong();
+    private AtomicLong openFileCount = new AtomicLong();
+    private AtomicLong closeFileCount = new AtomicLong();
 
     public DebugBufferCache(IBufferCache bufferCache) {
         this.bufferCache = bufferCache;
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceFactory.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceFactory.java
new file mode 100644
index 0000000..c443b0b
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceFactory.java
@@ -0,0 +1,19 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+public interface ILocalResourceFactory {
+    public LocalResource createLocalResource(long resourceId, String resourceName, int partition);
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceFactoryProvider.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceFactoryProvider.java
new file mode 100644
index 0000000..fb6a210
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceFactoryProvider.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import java.io.Serializable;
+
+public interface ILocalResourceFactoryProvider extends Serializable {
+    public ILocalResourceFactory getLocalResourceFactory();
+}
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
new file mode 100644
index 0000000..2dce39e
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ILocalResourceRepository {
+
+    public LocalResource getResourceById(long id) throws HyracksDataException;
+
+    public LocalResource getResourceByName(String name) throws HyracksDataException;
+
+    public void insert(LocalResource resource) throws HyracksDataException;
+
+    public void deleteResourceById(long id) 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/ILocalResourceRepositoryFactory.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepositoryFactory.java
new file mode 100644
index 0000000..4b3124a
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepositoryFactory.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ILocalResourceRepositoryFactory {
+    public ILocalResourceRepository createRepository() throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/LocalResource.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/LocalResource.java
new file mode 100644
index 0000000..f4575f3
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/LocalResource.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import java.io.Serializable;
+
+public class LocalResource implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final long resourceId;
+    private final String resourceName;
+    private final int partition;
+    private final int resourceType;
+    private final Object object;
+
+    public static final int TransientResource = 0;
+    public static final int LSMBTreeResource = 1;
+    public static final int LSMRTreeResource = 2;
+    public static final int LSMInvertedIndexResource = 3;
+
+    public LocalResource(long resourceId, String resourceName, int partition, int resourceType, Object object) {
+        this.resourceId = resourceId;
+        this.resourceName = resourceName;
+        this.partition = partition;
+        this.resourceType = resourceType;
+        this.object = object;
+    }
+
+    public long getResourceId() {
+        return resourceId;
+    }
+
+    public String getResourceName() {
+        return resourceName;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    public int getResourceType() {
+        return resourceType;
+    }
+
+    public Object getResourceObject() {
+        return object;
+    }
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/NoOpLocalResourceFactoryProvider.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/NoOpLocalResourceFactoryProvider.java
new file mode 100644
index 0000000..e69f4e7
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/NoOpLocalResourceFactoryProvider.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+/**
+ * Dummy operation factory provider that does nothing.
+ */
+public enum NoOpLocalResourceFactoryProvider implements ILocalResourceFactoryProvider {
+    INSTANCE;
+
+    @Override
+    public ILocalResourceFactory getLocalResourceFactory() {
+        return null;
+    }
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/PersistentLocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/PersistentLocalResourceRepository.java
new file mode 100644
index 0000000..1eedc78
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/PersistentLocalResourceRepository.java
@@ -0,0 +1,196 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class PersistentLocalResourceRepository implements ILocalResourceRepository {
+
+    private final List<String> mountPoints;
+    private static final String METADATA_FILE_NAME = ".metadata";
+    private Map<String, LocalResource> name2ResourceMap = new HashMap<String, LocalResource>();
+    private Map<Long, LocalResource> id2ResourceMap = new HashMap<Long, LocalResource>();
+
+    public PersistentLocalResourceRepository(List<String> mountPoints, String rootDir) throws HyracksDataException {
+        this.mountPoints = mountPoints;
+
+        File rootFile = new File(this.mountPoints.get(0), rootDir);
+        if (!rootFile.exists()) {
+            throw new HyracksDataException(rootFile.getAbsolutePath() + "doesn't exist.");
+        }
+
+        FilenameFilter filter = new FilenameFilter() {
+            public boolean accept(File dir, String name) {
+                if (name.equalsIgnoreCase(METADATA_FILE_NAME)) {
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        File[] childFileList = rootFile.listFiles();
+        if (childFileList == null) {
+        	return;
+        }
+        for (File childFile : childFileList) {
+            if (childFile.isDirectory()) {
+                File[] targetFileList = childFile.listFiles(filter);
+                if (targetFileList == null) {
+                	return;
+                }
+                
+                for (File targetFile : targetFileList) {
+                    LocalResource localResource = readLocalResource(targetFile);
+                    id2ResourceMap.put(localResource.getResourceId(), localResource);
+                    name2ResourceMap.put(localResource.getResourceName(), localResource);
+                }
+            }
+        }
+    }
+
+    @Override
+    public LocalResource getResourceById(long id) throws HyracksDataException {
+        return id2ResourceMap.get(id);
+    }
+
+    @Override
+    public LocalResource getResourceByName(String name) throws HyracksDataException {
+        return name2ResourceMap.get(name);
+    }
+
+    @Override
+    public synchronized void insert(LocalResource resource) throws HyracksDataException {
+        long id = resource.getResourceId();
+
+        if (id2ResourceMap.containsKey(id)) {
+            throw new HyracksDataException("Duplicate resource");
+        }
+        id2ResourceMap.put(id, resource);
+        name2ResourceMap.put(resource.getResourceName(), resource);
+
+        FileOutputStream fos = null;
+        ObjectOutputStream oosToFos = null;
+        try {
+            fos = new FileOutputStream(getFileName(mountPoints.get(0), resource.getResourceName()));
+            oosToFos = new ObjectOutputStream(fos);
+            oosToFos.writeObject(resource);
+            oosToFos.flush();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        } finally {
+            if (oosToFos != null) {
+                try {
+                    oosToFos.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+            if (oosToFos == null && fos != null) {
+                try {
+                    fos.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        }
+    }
+
+    @Override
+    public synchronized void deleteResourceById(long id) throws HyracksDataException {
+        LocalResource resource = id2ResourceMap.get(id);
+        if (resource == null) {
+            throw new HyracksDataException("Resource doesn't exist");
+        }
+        id2ResourceMap.remove(id);
+        name2ResourceMap.remove(resource.getResourceName());
+        File file = new File(getFileName(mountPoints.get(0), resource.getResourceName()));
+        file.delete();
+    }
+
+    @Override
+    public synchronized void deleteResourceByName(String name) throws HyracksDataException {
+        LocalResource resource = name2ResourceMap.get(name);
+        if (resource == null) {
+            throw new HyracksDataException("Resource doesn't exist");
+        }
+        id2ResourceMap.remove(resource.getResourceId());
+        name2ResourceMap.remove(name);
+        File file = new File(getFileName(mountPoints.get(0), resource.getResourceName()));
+        file.delete();
+    }
+
+    @Override
+    public List<LocalResource> getAllResources() throws HyracksDataException {
+        List<LocalResource> resources = new ArrayList<LocalResource>();
+        for (LocalResource resource : id2ResourceMap.values()) {
+            resources.add(resource);
+        }
+        return resources;
+    }
+
+    private String getFileName(String mountPoint, String baseDir) {
+
+        String fileName = new String(mountPoint);
+
+        if (!baseDir.endsWith(System.getProperty("file.separator"))) {
+            baseDir += System.getProperty("file.separator");
+        }
+        fileName += baseDir + METADATA_FILE_NAME;
+
+        return fileName;
+    }
+
+    private LocalResource readLocalResource(File file) throws HyracksDataException {
+        FileInputStream fis = null;
+        ObjectInputStream oisFromFis = null;
+
+        try {
+            fis = new FileInputStream(file);
+            oisFromFis = new ObjectInputStream(fis);
+            LocalResource resource = (LocalResource) oisFromFis.readObject();
+            return resource;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        } finally {
+            if (oisFromFis != null) {
+                try {
+                    oisFromFis.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+            if (oisFromFis == null && fis != null) {
+                try {
+                    fis.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        }
+    }
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/PersistentLocalResourceRepositoryFactory.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/PersistentLocalResourceRepositoryFactory.java
new file mode 100644
index 0000000..a2d0fa3
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/PersistentLocalResourceRepositoryFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+
+public class PersistentLocalResourceRepositoryFactory implements ILocalResourceRepositoryFactory {
+    private static final String rootDir = "";
+    private IIOManager ioManager;
+
+    public PersistentLocalResourceRepositoryFactory(IIOManager ioManager) {
+        this.ioManager = ioManager;
+    }
+
+    @Override
+    public ILocalResourceRepository createRepository() throws HyracksDataException {
+        List<String> mountPoints = new ArrayList<String>();
+        List<IODeviceHandle> devices = ioManager.getIODevices();
+
+        for (IODeviceHandle dev : devices) {
+            mountPoints.add(dev.getPath().getPath());
+        }
+        return new PersistentLocalResourceRepository(mountPoints, rootDir);
+    }
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ResourceIdFactory.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ResourceIdFactory.java
new file mode 100644
index 0000000..d4ec01f
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ResourceIdFactory.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ResourceIdFactory {
+    private AtomicLong id = null;
+
+    public ResourceIdFactory(long initialValue) {
+        id = new AtomicLong(initialValue);
+    }
+
+    public long createId() {
+        return id.getAndIncrement();
+    }
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ResourceIdFactoryProvider.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ResourceIdFactoryProvider.java
new file mode 100644
index 0000000..62cdb36
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ResourceIdFactoryProvider.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ResourceIdFactoryProvider {
+    private ILocalResourceRepository localResourceRepository;
+
+    public ResourceIdFactoryProvider(ILocalResourceRepository localResourceRepository) {
+        this.localResourceRepository = localResourceRepository;
+    }
+
+    public ResourceIdFactory createResourceIdFactory() throws HyracksDataException {
+        List<LocalResource> localResources = localResourceRepository.getAllResources();
+        long largestResourceId = 0;
+        for (LocalResource localResource : localResources) {
+            if (largestResourceId < localResource.getResourceId()) {
+                largestResourceId = localResource.getResourceId();
+            }
+        }
+        return new ResourceIdFactory(largestResourceId);
+    }
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientFileMapManager.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientFileMapManager.java
new file mode 100644
index 0000000..e8085da
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientFileMapManager.java
@@ -0,0 +1,56 @@
+package edu.uci.ics.hyracks.storage.common.file;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+
+public class TransientFileMapManager implements IFileMapManager {
+    private static final long serialVersionUID = 1L;
+
+    private Map<Integer, FileReference> id2nameMap = new HashMap<Integer, FileReference>();
+    private Map<FileReference, Integer> name2IdMap = new HashMap<FileReference, Integer>();
+    private int idCounter = 0;
+
+    @Override
+    public FileReference lookupFileName(int fileId) throws HyracksDataException {
+        FileReference fRef = id2nameMap.get(fileId);
+        if (fRef == null) {
+            throw new HyracksDataException("No mapping found for id: " + fileId);
+        }
+        return fRef;
+    }
+
+    @Override
+    public int lookupFileId(FileReference fileRef) throws HyracksDataException {
+        Integer fileId = name2IdMap.get(fileRef);
+        if (fileId == null) {
+            throw new HyracksDataException("No mapping found for name: " + fileRef);
+        }
+        return fileId;
+    }
+
+    @Override
+    public boolean isMapped(FileReference fileRef) {
+        return name2IdMap.containsKey(fileRef);
+    }
+
+    @Override
+    public boolean isMapped(int fileId) {
+        return id2nameMap.containsKey(fileId);
+    }
+
+    @Override
+    public void unregisterFile(int fileId) throws HyracksDataException {
+        FileReference fileRef = id2nameMap.remove(fileId);
+        name2IdMap.remove(fileRef);
+    }
+
+    @Override
+    public void registerFile(FileReference fileRef) throws HyracksDataException {
+        Integer fileId = idCounter++;
+        id2nameMap.put(fileId, fileRef);
+        name2IdMap.put(fileRef, fileId);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceFactory.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceFactory.java
new file mode 100644
index 0000000..ffd3dff
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+public class TransientLocalResourceFactory implements ILocalResourceFactory {
+
+    @Override
+    public LocalResource createLocalResource(long resourceId, String resourceName, int partition) {
+        return new LocalResource(resourceId, resourceName, partition, LocalResource.TransientResource, null);
+    }
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceFactoryProvider.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceFactoryProvider.java
new file mode 100644
index 0000000..d9a51a3
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceFactoryProvider.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+public class TransientLocalResourceFactoryProvider implements ILocalResourceFactoryProvider {
+    
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public ILocalResourceFactory getLocalResourceFactory() {
+        return new TransientLocalResourceFactory();
+    }
+}
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
new file mode 100644
index 0000000..55bd807
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class TransientLocalResourceRepository implements ILocalResourceRepository {
+
+    private Map<String, LocalResource> name2ResourceMap = new HashMap<String, LocalResource>();
+    private Map<Long, LocalResource> id2ResourceMap = new HashMap<Long, LocalResource>();
+
+    @Override
+    public LocalResource getResourceById(long id) throws HyracksDataException {
+        return id2ResourceMap.get(id);
+    }
+
+    @Override
+    public LocalResource getResourceByName(String name) throws HyracksDataException {
+        return name2ResourceMap.get(name);
+    }
+
+    @Override
+    public synchronized void insert(LocalResource resource) throws HyracksDataException {
+        long id = resource.getResourceId();
+
+        if (id2ResourceMap.containsKey(id)) {
+            throw new HyracksDataException("Duplicate resource");
+        }
+        id2ResourceMap.put(id, resource);
+        name2ResourceMap.put(resource.getResourceName(), resource);
+    }
+
+    @Override
+    public synchronized void deleteResourceById(long id) throws HyracksDataException {
+        LocalResource resource = id2ResourceMap.get(id);
+        if (resource == null) {
+            throw new HyracksDataException("Resource doesn't exist");
+        }
+        id2ResourceMap.remove(id);
+        name2ResourceMap.remove(resource.getResourceName());
+    }
+
+    @Override
+    public synchronized void deleteResourceByName(String name) throws HyracksDataException {
+        LocalResource resource = name2ResourceMap.get(name);
+        if (resource == null) {
+            throw new HyracksDataException("Resource doesn't exist");
+        }
+        id2ResourceMap.remove(resource.getResourceId());
+        name2ResourceMap.remove(name);
+    }
+
+    @Override
+    public List<LocalResource> getAllResources() throws HyracksDataException {
+        List<LocalResource> resources = new ArrayList<LocalResource>();
+        for (LocalResource resource : id2ResourceMap.values()) {
+            resources.add(resource);
+        }
+        return resources;
+    }
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepositoryFactory.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepositoryFactory.java
new file mode 100644
index 0000000..a6b00a4
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepositoryFactory.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2012 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.common.file;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class TransientLocalResourceRepositoryFactory implements ILocalResourceRepositoryFactory {
+
+    @Override
+    public ILocalResourceRepository createRepository() throws HyracksDataException {
+        return new TransientLocalResourceRepository();
+    }
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/smi/TransientFileMapManager.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/smi/TransientFileMapManager.java
deleted file mode 100644
index e4ddc08..0000000
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/smi/TransientFileMapManager.java
+++ /dev/null
@@ -1,57 +0,0 @@
-package edu.uci.ics.hyracks.storage.common.smi;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
-
-public class TransientFileMapManager implements IFileMapManager {
-    private static final long serialVersionUID = 1L;
-
-    private Map<Integer, FileReference> id2nameMap = new HashMap<Integer, FileReference>();
-    private Map<FileReference, Integer> name2IdMap = new HashMap<FileReference, Integer>();
-    private int idCounter = 0;
-
-    @Override
-    public FileReference lookupFileName(int fileId) throws HyracksDataException {
-        FileReference fRef = id2nameMap.get(fileId);
-        if (fRef == null) {
-            throw new HyracksDataException("No mapping found for id: " + fileId);
-        }
-        return fRef;
-    }
-
-    @Override
-    public int lookupFileId(FileReference fileRef) throws HyracksDataException {
-        Integer fileId = name2IdMap.get(fileRef);
-        if (fileId == null) {
-            throw new HyracksDataException("No mapping found for name: " + fileRef);
-        }
-        return fileId;
-    }
-
-    @Override
-    public boolean isMapped(FileReference fileRef) {
-        return name2IdMap.containsKey(fileRef);
-    }
-
-    @Override
-    public boolean isMapped(int fileId) {
-        return id2nameMap.containsKey(fileId);
-    }
-
-    @Override
-    public void unregisterFile(int fileId) throws HyracksDataException {
-        FileReference fileRef = id2nameMap.remove(fileId);
-        name2IdMap.remove(fileRef);
-    }
-
-    @Override
-    public void registerFile(FileReference fileRef) throws HyracksDataException {
-        Integer fileId = idCounter++;
-        id2nameMap.put(fileId, fileRef);
-        name2IdMap.put(fileRef, fileId);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/pom.xml b/hyracks/hyracks-test-support/pom.xml
index 89233c9..f2ae221 100644
--- a/hyracks/hyracks-test-support/pom.xml
+++ b/hyracks/hyracks-test-support/pom.xml
@@ -1,8 +1,6 @@
 <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>hyracks-test-support</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-test-support</name>
 
   <parent>
@@ -54,7 +52,7 @@
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-storage-am-invertedindex</artifactId>
+  		<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
   		<version>0.2.3-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java
new file mode 100644
index 0000000..30843aa
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java
@@ -0,0 +1,84 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+
+public abstract class AbstractModificationOperationCallbackTest extends AbstractOperationCallbackTest {
+
+    protected final ArrayTupleBuilder builder;
+    protected final ArrayTupleReference tuple;
+    protected final IModificationOperationCallback cb;
+
+    protected boolean isFoundNull;
+
+    public AbstractModificationOperationCallbackTest() {
+        this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+        this.tuple = new ArrayTupleReference();
+        this.cb = new VeriyfingModificationCallback();
+        this.isFoundNull = true;
+    }
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        super.tearDown();
+    }
+
+    @Test
+    public void modificationCallbackTest() throws Exception {
+        IIndexAccessor accessor = index.createAccessor(cb, NoOpOperationCallback.INSTANCE);
+
+        isFoundNull = true;
+        for (int i = 0; i < AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT; i++) {
+            TupleUtils.createIntegerTuple(builder, tuple, i);
+            accessor.insert(tuple);
+        }
+
+        isFoundNull = false;
+        for (int i = 0; i < AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT; i++) {
+            TupleUtils.createIntegerTuple(builder, tuple, i);
+            accessor.upsert(tuple);
+        }
+
+        isFoundNull = false;
+        for (int i = 0; i < AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT; i++) {
+            TupleUtils.createIntegerTuple(builder, tuple, i);
+            accessor.delete(tuple);
+        }
+    }
+
+    private class VeriyfingModificationCallback implements IModificationOperationCallback {
+
+        @Override
+        public void before(ITupleReference tuple) {
+            Assert.assertEquals(0, cmp.compare(AbstractModificationOperationCallbackTest.this.tuple, tuple));
+        }
+
+        @Override
+        public void found(ITupleReference before, ITupleReference after) {
+            if (isFoundNull) {
+                Assert.assertEquals(null, before);
+            } else {
+                Assert.assertEquals(0, cmp.compare(AbstractModificationOperationCallbackTest.this.tuple, before));
+            }
+            Assert.assertEquals(0, cmp.compare(AbstractModificationOperationCallbackTest.this.tuple, after));
+        }
+
+    }
+
+}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractOperationCallbackTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractOperationCallbackTest.java
new file mode 100644
index 0000000..41dfdfe
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractOperationCallbackTest.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+
+public abstract class AbstractOperationCallbackTest {
+    protected static final int NUM_KEY_FIELDS = 1;
+
+    @SuppressWarnings("rawtypes")
+    protected final ISerializerDeserializer[] keySerdes;
+    protected final MultiComparator cmp;
+    protected final int[] bloomFilterKeyFields;
+
+    protected IIndex index;
+
+    protected abstract void createIndexInstance() throws Exception;
+
+    public AbstractOperationCallbackTest() {
+        this.keySerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE };
+        this.cmp = MultiComparator.create(SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length));
+        bloomFilterKeyFields = new int[NUM_KEY_FIELDS];
+        for (int i = 0; i < NUM_KEY_FIELDS; ++i) {
+            bloomFilterKeyFields[i] = i;
+        }
+    }
+
+    public void setup() throws Exception {
+        createIndexInstance();
+        index.create();
+        index.activate();
+    }
+
+    public void tearDown() throws Exception {
+        index.deactivate();
+        index.destroy();
+    }
+}
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
new file mode 100644
index 0000000..ffa7c9a
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
@@ -0,0 +1,230 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+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.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+
+public abstract class AbstractSearchOperationCallbackTest extends AbstractOperationCallbackTest {
+    private static final int NUM_TASKS = 2;
+
+    protected final Lock lock;
+    protected final Condition condition;
+
+    protected ExecutorService executor;
+    protected boolean insertTaskStarted;
+
+    public AbstractSearchOperationCallbackTest() {
+        this.lock = new ReentrantLock(true);
+        this.condition = lock.newCondition();
+        this.insertTaskStarted = false;
+    }
+
+    @Before
+    public void setup() throws Exception {
+        executor = Executors.newFixedThreadPool(NUM_TASKS);
+        super.setup();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        executor.shutdown();
+        super.tearDown();
+    }
+
+    @Test
+    public void searchCallbackTest() throws Exception {
+        Future<Boolean> insertFuture = executor.submit(new InsertionTask());
+        Future<Boolean> searchFuture = executor.submit(new SearchTask());
+        Assert.assertTrue(searchFuture.get());
+        Assert.assertTrue(insertFuture.get());
+    }
+
+    private class SearchTask implements Callable<Boolean> {
+        private final ISearchOperationCallback cb;
+        private final IIndexAccessor accessor;
+        private final IIndexCursor cursor;
+        private final RangePredicate predicate;
+        private final ArrayTupleBuilder builder;
+        private final ArrayTupleReference tuple;
+
+        private boolean blockOnHigh;
+        private int blockingValue;
+        private int expectedAfterBlock;
+
+        public SearchTask() {
+            this.cb = new SynchronizingSearchOperationCallback();
+            this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb);
+            this.cursor = accessor.createSearchCursor();
+            this.predicate = new RangePredicate();
+            this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+            this.tuple = new ArrayTupleReference();
+
+            this.blockOnHigh = false;
+            this.blockingValue = -1;
+            this.expectedAfterBlock = -1;
+        }
+
+        @Override
+        public Boolean call() throws Exception {
+            lock.lock();
+            try {
+                if (!insertTaskStarted) {
+                    condition.await();
+                }
+
+                // begin a search on [101, +inf), blocking on 101
+                TupleUtils.createIntegerTuple(builder, tuple, 101);
+                predicate.setLowKey(tuple, true);
+                predicate.setHighKey(null, true);
+                accessor.search(cursor, predicate);
+                consumeIntTupleRange(101, 101, true, 101);
+
+                // consume tuples [102, 152], blocking on 151
+                consumeIntTupleRange(102, 151, true, 152);
+
+                // consume tuples [153, 300]
+                consumeIntTupleRange(153, 300, false, -1);
+
+                cursor.close();
+            } finally {
+                lock.unlock();
+            }
+
+            return true;
+        }
+
+        private void consumeIntTupleRange(int begin, int end, boolean blockOnHigh, int expectedAfterBlock)
+                throws Exception {
+            if (end < begin) {
+                throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
+            }
+
+            for (int i = begin; i <= end; i++) {
+                if (blockOnHigh == true && i == end) {
+                    this.blockOnHigh = true;
+                    this.blockingValue = end;
+                    this.expectedAfterBlock = expectedAfterBlock;
+                }
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                if (!cursor.hasNext()) {
+                    Assert.fail("Failed to consume entire tuple range since cursor is exhausted.");
+                }
+                cursor.next();
+
+                if (this.blockOnHigh) {
+                    TupleUtils.createIntegerTuple(builder, tuple, expectedAfterBlock);
+                }
+                Assert.assertEquals(0, cmp.compare(tuple, cursor.getTuple()));
+            }
+        }
+
+        private class SynchronizingSearchOperationCallback implements ISearchOperationCallback {
+
+            @Override
+            public boolean proceed(ITupleReference tuple) {
+                Assert.assertEquals(0, cmp.compare(SearchTask.this.tuple, tuple));
+                return false;
+            }
+
+            @Override
+            public void reconcile(ITupleReference tuple) {
+                Assert.assertEquals(0, cmp.compare(SearchTask.this.tuple, tuple));
+                if (blockOnHigh) {
+                    try {
+                        TupleUtils.createIntegerTuple(builder, SearchTask.this.tuple, expectedAfterBlock);
+                    } catch (HyracksDataException e) {
+                        e.printStackTrace();
+                    }
+                    condition.signal();
+                    condition.awaitUninterruptibly();
+                    blockOnHigh = false;
+                }
+            }
+
+            @Override
+            public void cancel(ITupleReference tuple) {
+                try {
+                    TupleUtils.createIntegerTuple(builder, SearchTask.this.tuple, blockingValue);
+                    Assert.assertEquals(0, cmp.compare(tuple, SearchTask.this.tuple));
+                    TupleUtils.createIntegerTuple(builder, SearchTask.this.tuple, expectedAfterBlock);
+                } catch (HyracksDataException e) {
+                    e.printStackTrace();
+                }
+            }
+
+        }
+    }
+
+    private class InsertionTask implements Callable<Boolean> {
+        private final IIndexAccessor accessor;
+        private final ArrayTupleBuilder builder;
+        private final ArrayTupleReference tuple;
+
+        public InsertionTask() {
+            this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+            this.tuple = new ArrayTupleReference();
+        }
+
+        @Override
+        public Boolean call() throws Exception {
+            lock.lock();
+            try {
+                insertTaskStarted = true;
+
+                // insert tuples [101, 200]
+                insertIntTupleRange(101, 200);
+                condition.signal();
+                condition.await();
+
+                // insert tuples [1, 100]
+                insertIntTupleRange(1, 100);
+                condition.signal();
+                condition.await();
+
+                // insert tuples [201, 300] and delete tuple 151
+                insertIntTupleRange(201, 300);
+                TupleUtils.createIntegerTuple(builder, tuple, 151);
+                accessor.delete(tuple);
+                condition.signal();
+            } finally {
+                lock.unlock();
+            }
+
+            return true;
+        }
+
+        private void insertIntTupleRange(int begin, int end) throws Exception {
+            if (end < begin) {
+                throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
+            }
+
+            for (int i = begin; i <= end; i++) {
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                accessor.insert(tuple);
+            }
+        }
+
+    }
+
+}
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 85bfdd2..6d7ee4d 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
@@ -38,6 +38,8 @@
             ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
             throws Exception {
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        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
@@ -55,9 +57,12 @@
                 orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
             }
         }
-        ctx.getIndex().close();
-    }
 
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+    
     @Override
     protected String getTestOpName() {
         return "BulkLoad";
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexDeleteTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexDeleteTest.java
index 93075a1..b96f252 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexDeleteTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexDeleteTest.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexDeleteTest extends OrderedIndexTestDriver {
@@ -31,14 +32,16 @@
         this.orderedIndexTestUtils = new OrderedIndexTestUtils();
     }
 
-    private static final int numInsertRounds = 3;
-    private static final int numDeleteRounds = 3;
+    private static final int numInsertRounds = AccessMethodTestsConfig.BTREE_NUM_INSERT_ROUNDS;
+    private static final int numDeleteRounds = AccessMethodTestsConfig.BTREE_NUM_DELETE_ROUNDS;
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, BTreeLeafFrameType leafType,
             ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
             throws Exception {
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         for (int i = 0; i < numInsertRounds; i++) {
             // We assume all fieldSerdes are of the same type. Check the first
             // one to determine which field types to generate.
@@ -60,7 +63,10 @@
                 }
             }
         }
-        ctx.getIndex().close();
+
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
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 a29be89..970526e 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
@@ -15,6 +15,8 @@
 
 package edu.uci.ics.hyracks.storage.am.btree;
 
+import static org.junit.Assert.fail;
+
 import java.util.Random;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -35,14 +37,16 @@
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 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.TestOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 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.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.UnsortedInputException;
+import edu.uci.ics.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
 @SuppressWarnings("rawtypes")
@@ -50,17 +54,13 @@
     protected static final Logger LOGGER = Logger.getLogger(OrderedIndexExamplesTest.class.getName());
     protected final Random rnd = new Random(50);
 
-    protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories)
-            throws TreeIndexException;
+    protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields) throws TreeIndexException;
 
-    protected abstract int getIndexFileId();
-    
     /**
-     * Fixed-Length Key,Value Example.
-     * 
-     * Create a tree index with one fixed-length key field and one fixed-length value
-     * field. Fill index with random values using insertions (not bulk load).
-     * Perform scans and range search.
+     * Fixed-Length Key,Value Example. Create a tree index with one fixed-length
+     * key field and one fixed-length value field. Fill index with random values
+     * using insertions (not bulk load). Perform scans and range search.
      */
     @Test
     public void fixedLengthKeyValueExample() throws Exception {
@@ -82,10 +82,13 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -93,7 +96,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         int numInserts = 10000;
         for (int i = 0; i < numInserts; i++) {
             int f0 = rnd.nextInt() % numInserts;
@@ -129,15 +133,92 @@
 
         rangeSearch(cmpFactories, indexAccessor, fieldSerdes, lowKey, highKey);
 
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Composite Key Example (Non-Unique Index).
-     * 
-     * Create a tree index with two fixed-length key fields and one fixed-length
-     * value field. Fill index with random values using insertions (not bulk
-     * load) Perform scans and range search.
+     * This test the btree page split. Originally this test didn't pass since
+     * the btree was spliting by cardinality and not size. Thus, we might end
+     * up with a situation where there is not enough space to insert the new
+     * tuple after the split which will throw an error and the split won't be
+     * propagated to upper level; thus, the tree is corrupted. Now, it split
+     * page by size. The correct behavior on abnormally large keys/values.
+     */
+    @Test
+    public void pageSplitTestExample() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BTree page split test.");
+        }
+
+        // Declare fields.
+        int fieldCount = 2;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        // Declare field serdes.
+        ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+
+        // Declare keys.
+        int keyFieldCount = 1;
+        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+        cmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
+
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
+
+        String key = "111";
+        String data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "222";
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "333";
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "444";
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "555";
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "666";
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
+    }
+
+    /**
+     * Composite Key Example (Non-Unique Index). Create a tree index with two
+     * fixed-length key fields and one fixed-length value field. Fill index with
+     * random values using insertions (not bulk load) Perform scans and range
+     * search.
      */
     @Test
     public void twoFixedLengthKeysOneFixedLengthValueExample() throws Exception {
@@ -161,10 +242,14 @@
         cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
         cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+        bloomFilterKeyFields[1] = 1;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -172,7 +257,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         int numInserts = 10000;
         for (int i = 0; i < 10000; i++) {
             int f0 = rnd.nextInt() % 2000;
@@ -210,7 +296,9 @@
         // Prefix-Range search in [-3, 3]
         rangeSearch(cmpFactories, indexAccessor, fieldSerdes, lowKey, highKey);
 
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
@@ -238,10 +326,13 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -249,7 +340,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         // Max string length to be generated.
         int maxLength = 10;
         int numInserts = 10000;
@@ -259,7 +351,7 @@
             TupleUtils.createTuple(tb, tuple, fieldSerdes, f0, f1);
             if (LOGGER.isLoggable(Level.INFO)) {
                 if (i % 1000 == 0) {
-                    LOGGER.info("Inserting " + f0 + " " + f1);
+                    LOGGER.info("Inserting[" + i + "] " + f0 + " " + f1);
                 }
             }
             try {
@@ -287,15 +379,16 @@
 
         rangeSearch(cmpFactories, indexAccessor, fieldSerdes, lowKey, highKey);
 
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Deletion Example.
-     * 
-     * Create a BTree with one variable-length key field and one variable-length
-     * value field. Fill B-tree with random values using insertions, then delete
-     * entries one-by-one. Repeat procedure a few times on same BTree.
+     * Deletion Example. Create a BTree with one variable-length key field and
+     * one variable-length value field. Fill B-tree with random values using
+     * insertions, then delete entries one-by-one. Repeat procedure a few times
+     * on same BTree.
      */
     @Test
     public void deleteExample() throws Exception {
@@ -317,14 +410,18 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         // Max string length to be generated.
         int runs = 3;
         for (int run = 0; run < runs; run++) {
@@ -388,15 +485,16 @@
                 break;
             }
         }
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Update example.
-     * 
-     * Create a BTree with one variable-length key field and one variable-length
-     * value field. Fill B-tree with random values using insertions, then update
-     * entries one-by-one. Repeat procedure a few times on same BTree.
+     * Update example. Create a BTree with one variable-length key field and one
+     * variable-length value field. Fill B-tree with random values using
+     * insertions, then update entries one-by-one. Repeat procedure a few times
+     * on same BTree.
      */
     @Test
     public void updateExample() throws Exception {
@@ -418,15 +516,19 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Inserting into tree...");
         }
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
         int maxLength = 10;
@@ -474,15 +576,14 @@
             // Do another scan after a round of updates.
             orderedScan(indexAccessor, fieldSerdes);
         }
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Bulk load example.
-     * 
-     * Load a tree with 100,000 tuples. BTree has a composite key to "simulate"
-     * non-unique index creation.
-     * 
+     * Bulk load example. Load a tree with 100,000 tuples. BTree has a composite
+     * key to "simulate" non-unique index creation.
      */
     @Test
     public void bulkLoadExample() throws Exception {
@@ -505,10 +606,14 @@
         cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
         cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+        bloomFilterKeyFields[1] = 1;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         // Load sorted records.
         int ins = 100000;
@@ -516,20 +621,21 @@
             LOGGER.info("Bulk loading " + ins + " tuples");
         }
         long start = System.currentTimeMillis();
-        IIndexBulkLoadContext bulkLoadCtx = treeIndex.beginBulkLoad(0.7f);
+        IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, ins);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
         for (int i = 0; i < ins; i++) {
             TupleUtils.createIntegerTuple(tb, tuple, i, i, 5);
-            treeIndex.bulkLoadAddTuple(tuple, bulkLoadCtx);
+            bulkLoader.add(tuple);
         }
-        treeIndex.endBulkLoad(bulkLoadCtx);
+        bulkLoader.end();
         long end = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(ins + " tuples loaded in " + (end - start) + "ms");
         }
 
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
 
         // Build low key.
         ArrayTupleBuilder lowKeyTb = new ArrayTupleBuilder(1);
@@ -544,15 +650,85 @@
         // Prefix-Range search in [44444, 44500]
         rangeSearch(cmpFactories, indexAccessor, fieldSerdes, lowKey, highKey);
 
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
-    private void orderedScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes)
-            throws Exception {
+    /**
+     * Bulk load failure example. Repeatedly loads a tree with 1,000 tuples, of
+     * which one tuple at each possible position does not conform to the
+     * expected order. We expect the bulk load to fail with an exception.
+     */
+    @Test
+    public void bulkOrderVerificationExample() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Bulk load order verification example");
+        }
+        // Declare fields.
+        int fieldCount = 2;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+
+        // declare keys
+        int keyFieldCount = 1;
+        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
+        Random rnd = new Random();
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        int ins = 1000;
+        for (int i = 1; i < ins; i++) {
+            ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+            treeIndex.create();
+            treeIndex.activate();
+
+            // Load sorted records, and expect to fail at tuple i.
+            IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, true, ins);
+            for (int j = 0; j < ins; j++) {
+                if (j > i) {
+                    fail("Bulk load failure test unexpectedly succeeded past tuple: " + j);
+                }
+                int key = j;
+                if (j == i) {
+                    int swapElementCase = Math.abs(rnd.nextInt()) % 2;
+                    if (swapElementCase == 0) {
+                        // Element equal to previous element.
+                        key--;
+                    } else {
+                        // Element smaller than previous element.
+                        key -= Math.abs(Math.random() % (ins - 1)) + 1;
+                    }
+                }
+                TupleUtils.createIntegerTuple(tb, tuple, key, 5);
+                try {
+                    bulkLoader.add(tuple);
+                } catch (UnsortedInputException e) {
+                    if (j != i) {
+                        fail("Unexpected exception: " + e.getMessage());
+                    }
+                    // Success.
+                    break;
+                }
+            }
+
+            treeIndex.deactivate();
+            treeIndex.destroy();
+        }
+    }
+
+    private void orderedScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Ordered Scan:");
         }
-        IIndexCursor scanCursor = (IIndexCursor) indexAccessor.createSearchCursor();        
+        IIndexCursor scanCursor = (IIndexCursor) indexAccessor.createSearchCursor();
         RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
         indexAccessor.search(scanCursor, nullPred);
         try {
@@ -569,45 +745,44 @@
         }
     }
 
-	private void diskOrderScan(IIndexAccessor indexAccessor,
-			ISerializerDeserializer[] fieldSerdes) throws Exception {
-		try {
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Disk-Order Scan:");
-			}
-			ITreeIndexAccessor treeIndexAccessor = (ITreeIndexAccessor) indexAccessor;
-			TreeDiskOrderScanCursor diskOrderCursor = (TreeDiskOrderScanCursor) treeIndexAccessor
-					.createDiskOrderScanCursor();
-			treeIndexAccessor.diskOrderScan(diskOrderCursor);
-			try {
-				while (diskOrderCursor.hasNext()) {
-					diskOrderCursor.next();
-					ITupleReference frameTuple = diskOrderCursor.getTuple();
-					String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
-					if (LOGGER.isLoggable(Level.INFO)) {
-						LOGGER.info(rec);
-					}
-				}
-			} finally {
-				diskOrderCursor.close();
-			}
-		} catch (UnsupportedOperationException e) {
-			// Ignore exception because some indexes, e.g. the LSMBTree, don't
-			// support disk-order scan.
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Ignoring disk-order scan since it's not supported.");
-			}
-		} catch (ClassCastException e) {
-			// Ignore exception because IIndexAccessor sometimes isn't
-			// an ITreeIndexAccessor, e.g., for the LSMBTree.
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Ignoring disk-order scan since it's not supported.");
-			}
-		}
-	}
+    private void diskOrderScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes) throws Exception {
+        try {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Disk-Order Scan:");
+            }
+            ITreeIndexAccessor treeIndexAccessor = (ITreeIndexAccessor) indexAccessor;
+            TreeIndexDiskOrderScanCursor diskOrderCursor = (TreeIndexDiskOrderScanCursor) treeIndexAccessor
+                    .createDiskOrderScanCursor();
+            treeIndexAccessor.diskOrderScan(diskOrderCursor);
+            try {
+                while (diskOrderCursor.hasNext()) {
+                    diskOrderCursor.next();
+                    ITupleReference frameTuple = diskOrderCursor.getTuple();
+                    String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info(rec);
+                    }
+                }
+            } finally {
+                diskOrderCursor.close();
+            }
+        } catch (UnsupportedOperationException e) {
+            // Ignore exception because some indexes, e.g. the LSMBTree, don't
+            // support disk-order scan.
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring disk-order scan since it's not supported.");
+            }
+        } catch (ClassCastException e) {
+            // Ignore exception because IIndexAccessor sometimes isn't
+            // an ITreeIndexAccessor, e.g., for the LSMBTree.
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring disk-order scan since it's not supported.");
+            }
+        }
+    }
 
-    private void rangeSearch(IBinaryComparatorFactory[] cmpFactories, IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes,
-            ITupleReference lowKey, ITupleReference highKey) throws Exception {
+    private void rangeSearch(IBinaryComparatorFactory[] cmpFactories, IIndexAccessor indexAccessor,
+            ISerializerDeserializer[] fieldSerdes, ITupleReference lowKey, ITupleReference highKey) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             String lowKeyString = TupleUtils.printTuple(lowKey, fieldSerdes);
             String highKeyString = TupleUtils.printTuple(highKey, fieldSerdes);
@@ -616,8 +791,7 @@
         ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
         MultiComparator lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, lowKey);
         MultiComparator highKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, highKey);
-        RangePredicate rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp,
-                highKeySearchCmp);
+        RangePredicate rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp);
         indexAccessor.search(rangeCursor, rangePred);
         try {
             while (rangeCursor.hasNext()) {
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexInsertTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexInsertTest.java
index d12603b..32b597c 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexInsertTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexInsertTest.java
@@ -24,12 +24,10 @@
 /**
  * Tests the BTree insert operation with strings and integer fields using
  * various numbers of key and payload fields.
- * 
  * Each tests first fills a BTree with randomly generated tuples. We compare the
  * following operations against expected results: 1. Point searches for all
  * tuples. 2. Ordered scan. 3. Disk-order scan. 4. Range search (and prefix
  * search for composite keys).
- * 
  */
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexInsertTest extends OrderedIndexTestDriver {
@@ -46,6 +44,8 @@
             ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
             throws Exception {
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         // 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) {
@@ -62,7 +62,10 @@
         if (prefixLowKey != null && prefixHighKey != null) {
             orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
         }
-        ctx.getIndex().close();
+
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java
index 3a894a2..fa22f6b 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java
@@ -29,95 +29,98 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.IndexMultiThreadTestDriver;
 import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
-import edu.uci.ics.hyracks.storage.am.common.TreeIndexMultiThreadTestDriver;
-import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 
 @SuppressWarnings("rawtypes")
-public abstract class OrderedIndexMultiThreadTest {    
-    
+public abstract class OrderedIndexMultiThreadTest {
+
     protected final Logger LOGGER = Logger.getLogger(OrderedIndexMultiThreadTest.class.getName());
-    
+
     // Machine-specific number of threads to use for testing.
     protected final int REGULAR_NUM_THREADS = Runtime.getRuntime().availableProcessors();
     // Excessive number of threads for testing.
     protected final int EXCESSIVE_NUM_THREADS = Runtime.getRuntime().availableProcessors() * 4;
-    protected final int NUM_OPERATIONS = 10000;
-    
-    protected ArrayList<TestWorkloadConf> workloadConfs = getTestWorkloadConf();    
-    
-    protected abstract void setUp() throws HyracksException;
-    
-    protected abstract void tearDown() throws HyracksDataException;        
+    protected final int NUM_OPERATIONS = AccessMethodTestsConfig.BTREE_MULTITHREAD_NUM_OPERATIONS;
 
-    protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories) throws TreeIndexException;
-    
-    protected abstract int getFileId();
-    
-    protected abstract ITreeIndexTestWorkerFactory getWorkerFactory();
-    
+    protected ArrayList<TestWorkloadConf> workloadConfs = getTestWorkloadConf();
+
+    protected abstract void setUp() throws HyracksException;
+
+    protected abstract void tearDown() throws HyracksDataException;
+
+    protected abstract IIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields) throws TreeIndexException;
+
+    protected abstract IIndexTestWorkerFactory getWorkerFactory();
+
     protected abstract ArrayList<TestWorkloadConf> getTestWorkloadConf();
-    
+
     protected abstract String getIndexTypeName();
-    
-    protected static float[] getUniformOpProbs(TestOperation[] ops) {
-        float[] opProbs = new float[ops.length];
-        for (int i = 0; i < ops.length; i++) {
-            opProbs[i] = 1.0f / (float) ops.length;
-        }
-        return opProbs;
-    }
-    
-    protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, int numThreads, TestWorkloadConf conf, String dataMsg) throws InterruptedException, TreeIndexException, HyracksException {
+
+    protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, int numThreads, TestWorkloadConf conf,
+            String dataMsg) throws InterruptedException, TreeIndexException, HyracksException {
         setUp();
-        
+
         if (LOGGER.isLoggable(Level.INFO)) {
-        	String indexTypeName = getIndexTypeName();
-            LOGGER.info(indexTypeName + " MultiThread Test:\nData: " + dataMsg + "; Threads: " + numThreads + "; Workload: " + conf.toString() + ".");
+            String indexTypeName = getIndexTypeName();
+            LOGGER.info(indexTypeName + " MultiThread Test:\nData: " + dataMsg + "; Threads: " + numThreads
+                    + "; Workload: " + conf.toString() + ".");
         }
-        
+
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
-        IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeys);     
-        
-        ITreeIndex index = createTreeIndex(typeTraits, cmpFactories);
-        ITreeIndexTestWorkerFactory workerFactory = getWorkerFactory();
-        
+        IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeys);
+
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[numKeys];
+        for (int i = 0; i < numKeys; ++i) {
+            bloomFilterKeyFields[i] = i;
+        }
+
+        IIndex index = createIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        IIndexTestWorkerFactory workerFactory = getWorkerFactory();
+
         // 4 batches per thread.
         int batchSize = (NUM_OPERATIONS / numThreads) / 4;
-        
-        TreeIndexMultiThreadTestDriver driver = new TreeIndexMultiThreadTestDriver(index, workerFactory, fieldSerdes, conf.ops, conf.opProbs);
-        driver.init(getFileId());
+
+        IndexMultiThreadTestDriver driver = new IndexMultiThreadTestDriver(index, workerFactory, fieldSerdes, conf.ops,
+                conf.opProbs);
+        driver.init();
         long[] times = driver.run(numThreads, 1, NUM_OPERATIONS, batchSize);
+        index.validate();
         driver.deinit();
-        
+
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("BTree MultiThread Test Time: " + times[0] + "ms");
         }
-        
+
         tearDown();
     }
-    
+
     @Test
-    public void oneIntKeyAndValue() throws InterruptedException, TreeIndexException, HyracksException {        
-        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    public void oneIntKeyAndValue() throws InterruptedException, TreeIndexException, HyracksException {
+        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE };
         int numKeys = 1;
         String dataMsg = "One Int Key And Value";
-        
+
         for (TestWorkloadConf conf : workloadConfs) {
             runTest(fieldSerdes, numKeys, REGULAR_NUM_THREADS, conf, dataMsg);
             runTest(fieldSerdes, numKeys, EXCESSIVE_NUM_THREADS, conf, dataMsg);
         }
     }
-    
+
     @Test
-    public void oneStringKeyAndValue() throws InterruptedException, TreeIndexException, HyracksException {        
-        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+    public void oneStringKeyAndValue() throws InterruptedException, TreeIndexException, HyracksException {
+        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
         int numKeys = 1;
         String dataMsg = "One String Key And Value";
-        
+
         for (TestWorkloadConf conf : workloadConfs) {
             runTest(fieldSerdes, numKeys, REGULAR_NUM_THREADS, conf, dataMsg);
             runTest(fieldSerdes, numKeys, EXCESSIVE_NUM_THREADS, conf, dataMsg);
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 f75a1f1..4ab0e87 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
@@ -20,16 +20,16 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
-import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestContext;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.IndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 
 @SuppressWarnings("rawtypes")
-public abstract class OrderedIndexTestContext extends TreeIndexTestContext<CheckTuple> {
+public abstract class OrderedIndexTestContext extends IndexTestContext<CheckTuple> {
 
     protected final TreeSet<CheckTuple> checkTuples = new TreeSet<CheckTuple>();
 
-    public OrderedIndexTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
-        super(fieldSerdes, treeIndex);
+    public OrderedIndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) {
+        super(fieldSerdes, index);
     }
 
     public void upsertCheckTuple(CheckTuple checkTuple, Collection<CheckTuple> checkTuples) {
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestDriver.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestDriver.java
index 8daa5e0..ef2ee0b 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestDriver.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestDriver.java
@@ -27,12 +27,13 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexTestDriver {
     protected final Logger LOGGER = Logger.getLogger(OrderedIndexTestDriver.class.getName());
 
-    protected static final int numTuplesToInsert = 10000;
+    protected static final int numTuplesToInsert = AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT;
 
     protected abstract OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception;
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
index a053dde..04c64fe 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
@@ -7,8 +7,8 @@
 import java.io.DataInputStream;
 import java.util.Collection;
 import java.util.Iterator;
-import java.util.NavigableSet;
 import java.util.Random;
+import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -25,7 +25,7 @@
 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.CheckTuple;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestContext;
 import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
@@ -43,47 +43,36 @@
                     actual.getFieldLength(i));
             DataInput dataIn = new DataInputStream(inStream);
             Object actualObj = fieldSerdes[i].deserialize(dataIn);
-            if (!actualObj.equals(expected.get(i))) {
-                fail("Actual and expected fields do not match on field " + i + ".\nExpected: " + expected.get(i)
+            if (!actualObj.equals(expected.getField(i))) {
+                fail("Actual and expected fields do not match on field " + i + ".\nExpected: " + expected.getField(i)
                         + "\nActual  : " + actualObj);
             }
         }
     }
 
     @SuppressWarnings("unchecked")
-    // Create a new TreeSet containing the elements satisfying the prefix
-    // search.
+    // Create a new TreeSet containing the elements satisfying the prefix search.
     // Implementing prefix search by changing compareTo() in CheckTuple does not
     // work.
-    public static TreeSet<CheckTuple> getPrefixExpectedSubset(TreeSet<CheckTuple> checkTuples, CheckTuple lowKey,
+    public static SortedSet<CheckTuple> getPrefixExpectedSubset(TreeSet<CheckTuple> checkTuples, CheckTuple lowKey,
             CheckTuple highKey) {
-        TreeSet<CheckTuple> expectedSubset = new TreeSet<CheckTuple>();
-        Iterator<CheckTuple> iter = checkTuples.iterator();
-        while (iter.hasNext()) {
-            CheckTuple t = iter.next();
-            boolean geLowKey = true;
-            boolean leHighKey = true;
-            for (int i = 0; i < lowKey.getNumKeys(); i++) {
-                if (t.get(i).compareTo(lowKey.get(i)) < 0) {
-                    geLowKey = false;
-                    break;
-                }
-            }
-            for (int i = 0; i < highKey.getNumKeys(); i++) {
-                if (t.get(i).compareTo(highKey.get(i)) > 0) {
-                    leHighKey = false;
-                    break;
-                }
-            }
-            if (geLowKey && leHighKey) {
-                expectedSubset.add(t);
-            }
+        lowKey.setIsHighKey(false);
+        highKey.setIsHighKey(true);
+        CheckTuple low = checkTuples.ceiling(lowKey);
+        CheckTuple high = checkTuples.floor(highKey);
+        if (low == null || high == null) {
+            // Must be empty.
+            return new TreeSet<CheckTuple>();
         }
-        return expectedSubset;
+        if (high.compareTo(low) < 0) {
+            // Must be empty.
+            return new TreeSet<CheckTuple>();
+        }
+        return checkTuples.subSet(low, true, high, true);
     }
 
     @SuppressWarnings("unchecked")
-    public void checkRangeSearch(ITreeIndexTestContext ctx, ITupleReference lowKey, ITupleReference highKey,
+    public void checkRangeSearch(IIndexTestContext ctx, ITupleReference lowKey, ITupleReference highKey,
             boolean lowKeyInclusive, boolean highKeyInclusive) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Testing Range Search.");
@@ -99,7 +88,7 @@
         CheckTuple lowKeyCheck = createCheckTupleFromTuple(lowKey, ctx.getFieldSerdes(), lowKeyCmp.getKeyFieldCount());
         CheckTuple highKeyCheck = createCheckTupleFromTuple(highKey, ctx.getFieldSerdes(),
                 highKeyCmp.getKeyFieldCount());
-        NavigableSet<CheckTuple> expectedSubset = null;
+        SortedSet<CheckTuple> expectedSubset = null;
         if (lowKeyCmp.getKeyFieldCount() < ctx.getKeyFieldCount()
                 || highKeyCmp.getKeyFieldCount() < ctx.getKeyFieldCount()) {
             // Searching on a key prefix (low key or high key or both).
@@ -132,7 +121,7 @@
         }
     }
 
-    public void checkPointSearches(ITreeIndexTestContext ictx) throws Exception {
+    public void checkPointSearches(IIndexTestContext ictx) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Testing Point Searches On All Expected Keys.");
         }
@@ -177,7 +166,7 @@
     }
 
     @SuppressWarnings("unchecked")
-    public void insertStringTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void insertStringTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         String[] fieldValues = new String[fieldCount];
@@ -207,10 +196,10 @@
             }
         }
     }
-    
-    public void upsertStringTuples(ITreeIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
-    	OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
-    	int fieldCount = ctx.getFieldCount();
+
+    public void upsertStringTuples(IIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
+        OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
+        int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         String[] fieldValues = new String[fieldCount];
         for (int i = 0; i < numTuples; i++) {
@@ -235,7 +224,7 @@
     }
 
     @SuppressWarnings("unchecked")
-    public void bulkLoadStringTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void bulkLoadStringTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         String[] fieldValues = new String[fieldCount];
@@ -262,9 +251,9 @@
         }
     }
 
-    public void upsertIntTuples(ITreeIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
+    public void upsertIntTuples(IIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
         OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
-    	int fieldCount = ctx.getFieldCount();
+        int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         int[] fieldValues = new int[ctx.getFieldCount()];
         // Scale range of values according to number of keys.
@@ -286,9 +275,9 @@
             ctx.upsertCheckTuple(createIntCheckTuple(fieldValues, ctx.getKeyFieldCount()), ctx.getCheckTuples());
         }
     }
-    
+
     @SuppressWarnings("unchecked")
-    public void updateTuples(ITreeIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
+    public void updateTuples(IIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
         OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
         int fieldCount = ctx.getFieldCount();
         int keyFieldCount = ctx.getKeyFieldCount();
@@ -317,7 +306,7 @@
             // Update check tuple's non-key fields.
             for (int j = keyFieldCount; j < fieldCount; j++) {
                 Comparable newValue = getRandomUpdateValue(ctx.getFieldSerdes()[j], rnd);
-                checkTuple.set(j, newValue);
+                checkTuple.setField(j, newValue);
             }
 
             createTupleFromCheckTuple(checkTuple, updateTupleBuilder, updateTuple, ctx.getFieldSerdes());
@@ -334,7 +323,7 @@
     public CheckTuple createStringCheckTuple(String[] fieldValues, int numKeyFields) {
         CheckTuple<String> checkTuple = new CheckTuple<String>(fieldValues.length, numKeyFields);
         for (String s : fieldValues) {
-            checkTuple.add((String) s);
+            checkTuple.appendField((String) s);
         }
         return checkTuple;
     }
@@ -396,7 +385,7 @@
     protected CheckTuple createIntCheckTuple(int[] fieldValues, int numKeyFields) {
         CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(fieldValues.length, numKeyFields);
         for (int v : fieldValues) {
-            checkTuple.add(v);
+            checkTuple.appendField(v);
         }
         return checkTuple;
     }
@@ -421,12 +410,12 @@
     }
 
     @Override
-    protected ArrayTupleBuilder createDeleteTupleBuilder(ITreeIndexTestContext ctx) {
+    protected ArrayTupleBuilder createDeleteTupleBuilder(IIndexTestContext ctx) {
         return new ArrayTupleBuilder(ctx.getKeyFieldCount());
     }
 
     @Override
-    protected boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple, ITreeIndexTestContext ctx)
+    protected boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple, IIndexTestContext ctx)
             throws HyracksDataException {
         @SuppressWarnings("unchecked")
         TreeSet<CheckTuple> checkTuples = (TreeSet<CheckTuple>) ctx.getCheckTuples();
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpdateTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpdateTest.java
index 65b2ade..049724e 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpdateTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpdateTest.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexUpdateTest extends OrderedIndexTestDriver {
@@ -31,7 +32,7 @@
         this.orderedIndexTestUtils = new OrderedIndexTestUtils();
     }
 
-    private static final int numUpdateRounds = 3;
+    private static final int numUpdateRounds = AccessMethodTestsConfig.BTREE_NUM_UPDATE_ROUNDS;
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, BTreeLeafFrameType leafType,
@@ -42,6 +43,8 @@
             return;
         }
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         // 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) {
@@ -60,6 +63,10 @@
                 orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
             }
         }
+
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpsertTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpsertTest.java
index 0d94a18..d34928f 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpsertTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpsertTest.java
@@ -24,12 +24,10 @@
 /**
  * Tests the BTree insert operation with strings and integer fields using
  * various numbers of key and payload fields.
- * 
  * Each tests first fills a BTree with randomly generated tuples. We compare the
  * following operations against expected results: 1. Point searches for all
  * tuples. 2. Ordered scan. 3. Disk-order scan. 4. Range search (and prefix
  * search for composite keys).
- * 
  */
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexUpsertTest extends OrderedIndexTestDriver {
@@ -46,6 +44,8 @@
             ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
             throws Exception {
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         // 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) {
@@ -62,7 +62,9 @@
         if (prefixLowKey != null && prefixHighKey != null) {
             orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
         }
-        ctx.getIndex().close();
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java
new file mode 100644
index 0000000..2226d94
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java
@@ -0,0 +1,96 @@
+package edu.uci.ics.hyracks.storage.am.common;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+
+public abstract class AbstractIndexLifecycleTest {
+
+    protected IIndex index;
+
+    protected abstract boolean persistentStateExists() throws Exception;
+
+    protected abstract boolean isEmptyIndex() throws Exception;
+
+    protected abstract void performInsertions() throws Exception;
+
+    protected abstract void checkInsertions() throws Exception;
+
+    protected abstract void clearCheckableInsertions() throws Exception;
+
+    @Before
+    public abstract void setup() throws Exception;
+
+    @After
+    public abstract void tearDown() throws Exception;
+
+    @Test
+    public void validSequenceTest() throws Exception {
+        // Double create is valid
+        index.create();
+        Assert.assertTrue(persistentStateExists());
+        index.create();
+        Assert.assertTrue(persistentStateExists());
+
+        // Double open is valid
+        index.activate();
+        index.activate();
+        Assert.assertTrue(isEmptyIndex());
+
+        // Insert some stuff
+        performInsertions();
+        checkInsertions();
+
+        // Check that the inserted stuff isn't there
+        clearCheckableInsertions();
+        index.clear();
+        Assert.assertTrue(isEmptyIndex());
+
+        // Insert more stuff
+        performInsertions();
+
+        // Double close is valid
+        index.deactivate();
+        index.deactivate();
+
+        // Check that the inserted stuff is still there
+        index.activate();
+        checkInsertions();
+        index.deactivate();
+
+        // Double destroy is valid
+        index.destroy();
+        Assert.assertFalse(persistentStateExists());
+        index.destroy();
+        Assert.assertFalse(persistentStateExists());
+    }
+
+    @Test(expected = HyracksDataException.class)
+    public void invalidSequenceTest1() throws Exception {
+        index.create();
+        index.activate();
+        index.create();
+    }
+
+    @Test(expected = HyracksDataException.class)
+    public void invalidSequenceTest2() throws Exception {
+        index.create();
+        index.activate();
+        index.destroy();
+    }
+
+    @Test(expected = HyracksDataException.class)
+    public void invalidSequenceTest3() throws Exception {
+        index.create();
+        index.clear();
+    }
+
+    @Test(expected = HyracksDataException.class)
+    public void invalidSequenceTest4() throws Exception {
+        index.clear();
+    }
+}
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
new file mode 100644
index 0000000..f9ff26a
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 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.common;
+
+import java.util.Random;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+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.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
+
+public abstract class AbstractIndexTestWorker extends Thread implements ITreeIndexTestWorker {
+    private final Random rnd;
+    private final DataGenThread dataGen;
+    private final TestOperationSelector opSelector;
+    private final int numBatches;
+
+    protected final IIndexAccessor indexAccessor;
+
+    public AbstractIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+        this.dataGen = dataGen;
+        this.opSelector = opSelector;
+        this.numBatches = numBatches;
+        this.rnd = new Random();
+        this.indexAccessor = index.createAccessor(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
+    }
+
+    @Override
+    public void run() {
+        try {
+            for (int i = 0; i < numBatches; i++) {
+                TupleBatch batch = dataGen.getBatch();
+                for (int j = 0; j < batch.size(); j++) {
+                    TestOperation op = opSelector.getOp(rnd.nextInt());
+                    ITupleReference tuple = batch.get(j);
+                    performOp(tuple, op);
+                }
+                dataGen.releaseBatch(batch);
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
+    protected void consumeCursorTuples(IIndexCursor cursor) throws HyracksDataException, IndexException {
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+            }
+        } finally {
+            cursor.close();
+        }
+    }
+}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractTreeIndexTestWorker.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractTreeIndexTestWorker.java
deleted file mode 100644
index eca9b35..0000000
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractTreeIndexTestWorker.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Copyright 2009-2010 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.common;
-
-import java.util.Random;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
-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.datagen.DataGenThread;
-import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
-
-public abstract class AbstractTreeIndexTestWorker extends Thread implements ITreeIndexTestWorker {
-    private Random rnd = new Random();
-    private final DataGenThread dataGen;
-    private final TestOperationSelector opSelector;
-    private final int numBatches;
-    
-    protected final IIndexAccessor indexAccessor;
-    
-    public AbstractTreeIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, ITreeIndex index, int numBatches) {
-        this.dataGen = dataGen;
-        this.opSelector = opSelector;
-        this.numBatches = numBatches;
-        indexAccessor = index.createAccessor();
-    }
-    
-    @Override
-    public void run() {
-        try {
-            for (int i = 0; i < numBatches; i++) {
-                TupleBatch batch = dataGen.getBatch();     
-                for (int j = 0; j < batch.size(); j++) {
-                    TestOperation op = opSelector.getOp(rnd.nextInt());
-                    ITupleReference tuple = batch.get(j);
-                    performOp(tuple, op);
-                }
-                dataGen.releaseBatch(batch);
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
-    }
-    
-    protected void consumeCursorTuples(IIndexCursor cursor) throws HyracksDataException {
-        try {
-            while (cursor.hasNext()) {
-                cursor.next();
-            }
-        } finally {
-            cursor.close();
-        }
-    }
-}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java
index 4b4b90b..b7037d6 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java
@@ -17,42 +17,75 @@
 
 @SuppressWarnings({"rawtypes", "unchecked"})
 public class CheckTuple<T extends Comparable<T>> implements Comparable<T> {
-    protected final int numKeys;    
-    protected final Comparable[] tuple;
+    protected final int numKeys;
+    protected final Comparable[] fields;
     protected int pos;
+    protected boolean isHighKey;
 
     public CheckTuple(int numFields, int numKeys) {
         this.numKeys = numKeys;
-        this.tuple = new Comparable[numFields];
+        this.fields = new Comparable[numFields];
         pos = 0;
+        isHighKey = false;
     }
 
-    public void add(T e) {
-        tuple[pos++] = e;
+    public void appendField(T e) {
+        fields[pos++] = e;
     }
 
-    @Override
-    public int compareTo(T o) {
-        CheckTuple<T> other = (CheckTuple<T>)o;
-        for (int i = 0; i < numKeys; i++) {            
-            int cmp = tuple[i].compareTo(other.get(i));
-            if (cmp != 0) {
-                return cmp;
-            }
+	@Override
+	public int compareTo(T o) {
+		CheckTuple<T> other = (CheckTuple<T>) o;
+		int cmpFieldCount = Math.min(other.getNumKeys(), numKeys);
+		for (int i = 0; i < cmpFieldCount; i++) {
+			int cmp = fields[i].compareTo(other.getField(i));
+			if (cmp != 0) {
+				return cmp;
+			}
+		}
+		if (other.getNumKeys() == numKeys) {
+		    return 0;
+		}
+		if (other.getNumKeys() < numKeys) {
+		    return (other.isHighKey) ? -1 : 1;
+		}
+		if (other.getNumKeys() > numKeys) {
+            return (isHighKey) ? 1 : -1;
         }
-        return 0;
-    }
+		return 0;
+	}
 
-    public T get(int idx) {
-        return (T)tuple[idx];
-    }
+	@Override
+	public boolean equals(Object o) {
+		if (!(o instanceof Comparable<?>)) {
+			return false;
+		}
+		return compareTo((T) o) == 0;
+	}
     
-    public void set(int idx, T e) {
-        tuple[idx] = e;
+	@Override
+	public int hashCode() {
+		int hash = 0;
+		for (int i = 0; i < numKeys; i++) {
+			hash = 37 * hash + fields[i].hashCode();
+		}
+		return hash;
+	}
+	
+	public void setIsHighKey(boolean isHighKey) {
+	    this.isHighKey = isHighKey;
+	}
+	
+	public T getField(int idx) {
+		return (T) fields[idx];
+	}
+    
+    public void setField(int idx, T e) {
+        fields[idx] = e;
     }
     
     public int size() {
-        return tuple.length;
+        return fields.length;
     }
     
     public int getNumKeys() {
@@ -62,9 +95,9 @@
     @Override
     public String toString() {
         StringBuilder strBuilder = new StringBuilder();
-        for (int i = 0; i < tuple.length; i++) {
-            strBuilder.append(tuple[i].toString());
-            if (i != tuple.length-1) {
+        for (int i = 0; i < fields.length; i++) {
+            strBuilder.append(fields[i].toString());
+            if (i != fields.length-1) {
                 strBuilder.append(" ");
             }
         }
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestContext.java
new file mode 100644
index 0000000..3599c5e
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestContext.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2010 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.common;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+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;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+
+@SuppressWarnings("rawtypes")
+public interface IIndexTestContext<T extends CheckTuple> {
+    public int getFieldCount();
+
+    public int getKeyFieldCount();
+
+    public ISerializerDeserializer[] getFieldSerdes();
+
+    public IBinaryComparatorFactory[] getComparatorFactories();
+
+    public IIndexAccessor getIndexAccessor();
+
+    public IIndex getIndex();
+
+    public ArrayTupleReference getTuple();
+
+    public ArrayTupleBuilder getTupleBuilder();
+
+    public void insertCheckTuple(T checkTuple, Collection<T> checkTuples);      
+
+    public void deleteCheckTuple(T checkTuple, Collection<T> checkTuples);
+
+    public Collection<T> getCheckTuples();
+
+}
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
new file mode 100644
index 0000000..d4efb3e
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 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.common;
+
+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);
+}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/ITreeIndexTestWorkerFactory.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/ITreeIndexTestWorkerFactory.java
deleted file mode 100644
index 64b5aea..0000000
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/ITreeIndexTestWorkerFactory.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Copyright 2009-2010 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.common;
-
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
-
-public interface ITreeIndexTestWorkerFactory {
-    public AbstractTreeIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, ITreeIndex index, int numBatches);
-}
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
new file mode 100644
index 0000000..ca1d28f
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2009-2010 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.common;
+
+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.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+
+@SuppressWarnings("rawtypes")
+public class IndexMultiThreadTestDriver {
+    protected static final int RANDOM_SEED = 50;
+    // Means no additional payload. Only the specified fields.
+    protected static final int PAYLOAD_SIZE = 0;
+    protected final TestOperationSelector opSelector;
+    protected final ISerializerDeserializer[] fieldSerdes;
+    protected final IIndex index;
+    protected final IIndexTestWorkerFactory workerFactory;
+
+    public IndexMultiThreadTestDriver(IIndex index, IIndexTestWorkerFactory workerFactory,
+            ISerializerDeserializer[] fieldSerdes, TestOperation[] ops, double[] opProbs) {
+        this.index = index;
+        this.workerFactory = workerFactory;
+        this.fieldSerdes = fieldSerdes;
+        this.opSelector = new TestOperationSelector(ops, opProbs);
+    }
+
+    public void init() throws HyracksDataException {
+        index.create();
+        index.activate();
+    }
+
+    public long[] run(int numThreads, int numRepeats, int numOps, int batchSize) throws InterruptedException,
+            TreeIndexException {
+        int numBatches = numOps / batchSize;
+        int threadNumBatches = numBatches / numThreads;
+        if (threadNumBatches <= 0) {
+            throw new TreeIndexException("Inconsistent parameters given. Need at least one batch per thread.");
+        }
+        long[] times = new long[numRepeats];
+        for (int i = 0; i < numRepeats; i++) {
+            DataGenThread dataGen = createDatagenThread(numThreads, numBatches, batchSize);
+            dataGen.start();
+            // Wait until the tupleBatchQueue is filled to capacity.
+            while (dataGen.tupleBatchQueue.remainingCapacity() != 0 && dataGen.tupleBatchQueue.size() != numBatches) {
+                Thread.sleep(10);
+            }
+
+            // Start worker threads.
+            AbstractIndexTestWorker[] workers = new AbstractIndexTestWorker[numThreads];
+            long start = System.currentTimeMillis();
+            for (int j = 0; j < numThreads; j++) {
+                workers[j] = workerFactory.create(dataGen, opSelector, index, threadNumBatches);
+                workers[j].start();
+            }
+            // Join worker threads.
+            for (int j = 0; j < numThreads; j++) {
+                workers[j].join();
+            }
+            long end = System.currentTimeMillis();
+            times[i] = end - start;
+        }
+        return times;
+    }
+
+    public void deinit() throws HyracksDataException {
+        index.deactivate();
+        index.destroy();
+    }
+
+    // To allow subclasses to override the data gen params.
+    public DataGenThread createDatagenThread(int numThreads, int numBatches, int batchSize) {
+        return new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, PAYLOAD_SIZE, RANDOM_SEED,
+                2 * numThreads, false);
+    }
+}
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
new file mode 100644
index 0000000..e0aa1db
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 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.common;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+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;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+
+@SuppressWarnings("rawtypes")
+public abstract class IndexTestContext<T extends CheckTuple> implements IIndexTestContext<T> {
+    protected final ISerializerDeserializer[] fieldSerdes;
+    protected final IIndex index;
+    protected final ArrayTupleBuilder tupleBuilder;
+    protected final ArrayTupleReference tuple = new ArrayTupleReference();
+    protected final IIndexAccessor indexAccessor;
+
+    public IndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) {
+        this.fieldSerdes = fieldSerdes;
+        this.index = index;
+        this.indexAccessor = (IIndexAccessor) index.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
+        this.tupleBuilder = new ArrayTupleBuilder(fieldSerdes.length);
+    }
+
+    @Override
+    public int getFieldCount() {
+        return fieldSerdes.length;
+    }
+
+    @Override
+    public IIndexAccessor getIndexAccessor() {
+        return indexAccessor;
+    }
+
+    @Override
+    public ArrayTupleReference getTuple() {
+        return tuple;
+    }
+
+    @Override
+    public ArrayTupleBuilder getTupleBuilder() {
+        return tupleBuilder;
+    }
+
+    @Override
+    public ISerializerDeserializer[] getFieldSerdes() {
+        return fieldSerdes;
+    }
+
+    @Override
+    public IIndex getIndex() {
+        return index;
+    }
+
+    @Override
+    public void insertCheckTuple(T checkTuple, Collection<T> checkTuples) {
+        checkTuples.add(checkTuple);
+    }
+
+    @Override
+    public void deleteCheckTuple(T checkTuple, Collection<T> checkTuples) {
+        checkTuples.remove(checkTuple);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationCallback.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationCallback.java
new file mode 100644
index 0000000..04f888b
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationCallback.java
@@ -0,0 +1,45 @@
+package edu.uci.ics.hyracks.storage.am.common;
+
+import java.util.Random;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+
+public enum TestOperationCallback implements ISearchOperationCallback, IModificationOperationCallback {
+    INSTANCE;
+
+    private static final int RANDOM_SEED = 50;
+    private final Random random = new Random();
+
+    private TestOperationCallback() {
+        random.setSeed(RANDOM_SEED);
+    }
+
+    @Override
+    public boolean proceed(ITupleReference tuple) {
+        // Always fail
+        return false;
+    }
+
+    @Override
+    public void reconcile(ITupleReference tuple) {
+        // Do nothing.
+    }
+
+    @Override
+    public void before(ITupleReference tuple) {
+        // Do nothing.        
+    }
+
+    @Override
+    public void found(ITupleReference before, ITupleReference after) {
+        // Do nothing.        
+    }
+
+    @Override
+    public void cancel(ITupleReference tuple) {
+        // Do nothing.
+    }
+
+}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationSelector.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationSelector.java
index 1ae79a1..47735e5 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationSelector.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationSelector.java
@@ -15,7 +15,8 @@
 
 package edu.uci.ics.hyracks.storage.am.common;
 
-import java.util.Arrays;
+import edu.uci.ics.hyracks.storage.am.common.datagen.ProbabilityHelper;
+
 
 public class TestOperationSelector {
 
@@ -32,15 +33,15 @@
     }
     
     private final TestOperation[] ops;
-    private final int[] opRanges;    
+    private final int[] cumulIntRanges;    
     
-    public TestOperationSelector(TestOperation[] ops, float[] opProbs) {
+    public TestOperationSelector(TestOperation[] ops, double[] opProbs) {
         sanityCheck(ops, opProbs);
         this.ops = ops;
-        this.opRanges = getOpRanges(opProbs);
+        this.cumulIntRanges = ProbabilityHelper.getCumulIntRanges(opProbs);
     }
     
-    private void sanityCheck(TestOperation[] ops, float[] opProbs) {
+    private void sanityCheck(TestOperation[] ops, double[] opProbs) {
         if (ops.length == 0) {
             throw new RuntimeException("Empty op array.");
         }
@@ -59,25 +60,8 @@
         }
     }
     
-    private int[] getOpRanges(float[] opProbabilities) {
-        int[] opRanges = new int[opProbabilities.length];
-        if (opRanges.length > 1) {
-            opRanges[0] = (int) Math.floor(Integer.MAX_VALUE * opProbabilities[0]);
-            for (int i = 1; i < opRanges.length - 1; i++) {
-                opRanges[i] = opRanges[i - 1] + (int) Math.floor(Integer.MAX_VALUE * opProbabilities[i]);
-            }
-            opRanges[opRanges.length - 1] = Integer.MAX_VALUE;
-        } else {
-            opRanges[0] = Integer.MAX_VALUE;
-        }
-        return opRanges;
-    }
-    
     public TestOperation getOp(int randomInt) {
-        int ix = Arrays.binarySearch(opRanges, randomInt);
-        if (ix < 0) {
-            ix = -ix - 1;
-        }
+        int ix = ProbabilityHelper.choose(cumulIntRanges, randomInt);
         return ops[ix];
     }
 }
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestWorkloadConf.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestWorkloadConf.java
index 2437514..28845ed 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestWorkloadConf.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestWorkloadConf.java
@@ -19,9 +19,9 @@
 
 public class TestWorkloadConf {
     public final TestOperation[] ops;
-    public final float[] opProbs;
+    public final double[] opProbs;
 
-    public TestWorkloadConf(TestOperation[] ops, float[] opProbs) {
+    public TestWorkloadConf(TestOperation[] ops, double[] opProbs) {
         this.ops = ops;
         this.opProbs = opProbs;
     }
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexMultiThreadTestDriver.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexMultiThreadTestDriver.java
deleted file mode 100644
index 8c1d06f..0000000
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexMultiThreadTestDriver.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Copyright 2009-2010 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.common;
-
-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.TestOperationSelector.TestOperation;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
-
-@SuppressWarnings("rawtypes")
-public class TreeIndexMultiThreadTestDriver {
-    private static final int RANDOM_SEED = 50;
-    // Means no additional payload. Only the specified fields.
-    private static final int PAYLOAD_SIZE = 0;
-    private final TestOperationSelector opSelector;    
-    private final ISerializerDeserializer[] fieldSerdes;
-    private final ITreeIndex index;
-    private final ITreeIndexTestWorkerFactory workerFactory;
-    
-    public TreeIndexMultiThreadTestDriver(ITreeIndex index, ITreeIndexTestWorkerFactory workerFactory,
-            ISerializerDeserializer[] fieldSerdes, TestOperation[] ops, float[] opProbs) {
-        this.index = index;
-        this.workerFactory = workerFactory;
-        this.fieldSerdes = fieldSerdes;
-        this.opSelector = new TestOperationSelector(ops, opProbs);
-    }      
-    
-    public void init(int fileId) throws HyracksDataException {
-    	index.create(fileId);
-    	index.open(fileId);
-    }
-    
-    public long[] run(int numThreads, int numRepeats, int numOps, int batchSize) throws InterruptedException, TreeIndexException {
-        int numBatches = numOps / batchSize;
-        int threadNumBatches = numBatches / numThreads;
-        if (threadNumBatches <= 0) {
-            throw new TreeIndexException("Inconsistent parameters given. Need at least one batch per thread.");
-        }
-        long[] times = new long[numRepeats];
-        for (int i = 0; i < numRepeats; i++) {
-            DataGenThread dataGen = createDatagenThread(numThreads, numBatches, batchSize);
-            dataGen.start();
-            // Wait until the tupleBatchQueue is filled to capacity.
-            while (dataGen.tupleBatchQueue.remainingCapacity() != 0 && dataGen.tupleBatchQueue.size() != numBatches) {
-                Thread.sleep(10);
-            }
-                        
-            // Start worker threads.
-            AbstractTreeIndexTestWorker[] workers = new AbstractTreeIndexTestWorker[numThreads];
-            long start = System.currentTimeMillis();
-            for (int j = 0; j < numThreads; j++) {
-                workers[j] = workerFactory.create(dataGen, opSelector, index, threadNumBatches);
-                workers[j].start();
-            }
-            // Join worker threads.
-            for (int j = 0; j < numThreads; j++) {                
-                workers[j].join();
-            }
-            long end = System.currentTimeMillis();
-            times[i] = end - start;
-        }
-        return times;
-    }
-    
-    public void deinit() throws HyracksDataException {
-    	index.close();
-    }
-    
-    // To allow subclasses to override the data gen params.
-    public DataGenThread createDatagenThread(int numThreads, int numBatches, int batchSize) {
-        return new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, PAYLOAD_SIZE, RANDOM_SEED, 2*numThreads, false);
-    }
-}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestContext.java
deleted file mode 100644
index bc5312c..0000000
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestContext.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Copyright 2009-2010 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.common;
-
-import java.util.Collection;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-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.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-
-@SuppressWarnings("rawtypes")
-public abstract class TreeIndexTestContext<T extends CheckTuple> implements ITreeIndexTestContext<T> {
-    protected final ISerializerDeserializer[] fieldSerdes;
-    protected final ITreeIndex treeIndex;
-    protected final ArrayTupleBuilder tupleBuilder;
-    protected final ArrayTupleReference tuple = new ArrayTupleReference();
-    protected final IIndexAccessor indexAccessor;
-
-    public TreeIndexTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
-        this.fieldSerdes = fieldSerdes;
-        this.treeIndex = treeIndex;
-        this.indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
-        this.tupleBuilder = new ArrayTupleBuilder(fieldSerdes.length);
-    }
-
-    @Override
-    public int getFieldCount() {
-        return fieldSerdes.length;
-    }
-
-    @Override
-    public IIndexAccessor getIndexAccessor() {
-        return indexAccessor;
-    }
-
-    @Override
-    public ArrayTupleReference getTuple() {
-        return tuple;
-    }
-
-    @Override
-    public ArrayTupleBuilder getTupleBuilder() {
-        return tupleBuilder;
-    }
-
-    @Override
-    public ISerializerDeserializer[] getFieldSerdes() {
-        return fieldSerdes;
-    }
-
-    @Override
-    public ITreeIndex getIndex() {
-        return treeIndex;
-    }
-
-    @Override
-    public void insertCheckTuple(T checkTuple, Collection<T> checkTuples) {
-        checkTuples.add(checkTuple);
-    }
-
-    @Override
-    public void deleteCheckTuple(T checkTuple, Collection<T> checkTuples) {
-        checkTuples.remove(checkTuple);
-    }
-}
\ No newline at end of file
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 d16553a..1a80231 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
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2010 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.common;
 
 import static org.junit.Assert.fail;
@@ -18,7 +33,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
@@ -44,11 +59,11 @@
 
     protected abstract Collection createCheckTuplesCollection();
 
-    protected abstract ArrayTupleBuilder createDeleteTupleBuilder(ITreeIndexTestContext ctx);
+    protected abstract ArrayTupleBuilder createDeleteTupleBuilder(IIndexTestContext ctx);
 
     // See if tuple with corresponding checkTuple exists in ctx.checkTuples.
     protected abstract boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple,
-            ITreeIndexTestContext ctx) throws HyracksDataException;
+            IIndexTestContext ctx) throws HyracksDataException;
 
     @SuppressWarnings("unchecked")
     public static void createTupleFromCheckTuple(CheckTuple checkTuple, ArrayTupleBuilder tupleBuilder,
@@ -57,7 +72,7 @@
         DataOutput dos = tupleBuilder.getDataOutput();
         tupleBuilder.reset();
         for (int i = 0; i < fieldCount; i++) {
-            fieldSerdes[i].serialize(checkTuple.get(i), dos);
+            fieldSerdes[i].serialize(checkTuple.getField(i), dos);
             tupleBuilder.addFieldEndOffset();
         }
         tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
@@ -73,13 +88,13 @@
                     tuple.getFieldLength(i));
             DataInput dataIn = new DataInputStream(inStream);
             Comparable fieldObj = (Comparable) fieldSerdes[i].deserialize(dataIn);
-            checkTuple.add(fieldObj);
+            checkTuple.appendField(fieldObj);
         }
         return checkTuple;
     }
 
     @SuppressWarnings("unchecked")
-    public void checkScan(ITreeIndexTestContext ctx) throws Exception {
+    public void checkScan(IIndexTestContext ctx) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Testing Scan.");
         }
@@ -90,7 +105,7 @@
         checkExpectedResults(scanCursor, ctx.getCheckTuples(), ctx.getFieldSerdes(), ctx.getKeyFieldCount(), checkIter);
     }
 
-    public void checkDiskOrderScan(ITreeIndexTestContext ctx) throws Exception {
+    public void checkDiskOrderScan(IIndexTestContext ctx) throws Exception {
         try {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Testing Disk-Order Scan.");
@@ -128,16 +143,16 @@
                 LOGGER.info("Ignoring disk-order scan since it's not supported.");
             }
         } catch (ClassCastException e) {
-			// Ignore exception because IIndexAccessor sometimes isn't
-			// an ITreeIndexAccessor, e.g., for the LSMBTree.
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Ignoring disk-order scan since it's not supported.");
-			}
-		}
+            // Ignore exception because IIndexAccessor sometimes isn't
+            // an ITreeIndexAccessor, e.g., for the LSMBTree.
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring disk-order scan since it's not supported.");
+            }
+        }
     }
 
     @SuppressWarnings("unchecked")
-    public void insertIntTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void insertIntTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         int[] fieldValues = new int[ctx.getFieldCount()];
@@ -165,9 +180,9 @@
             }
         }
     }
-    
+
     @SuppressWarnings("unchecked")
-    public void upsertIntTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void upsertIntTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         int[] fieldValues = new int[ctx.getFieldCount()];
@@ -197,7 +212,7 @@
     }
 
     @SuppressWarnings("unchecked")
-    public void bulkLoadIntTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void bulkLoadIntTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         int[] fieldValues = new int[ctx.getFieldCount()];
@@ -221,14 +236,14 @@
         }
     }
 
-    public static void bulkLoadCheckTuples(ITreeIndexTestContext ctx, Collection<CheckTuple> checkTuples)
+    public static void bulkLoadCheckTuples(IIndexTestContext ctx, Collection<CheckTuple> checkTuples)
             throws HyracksDataException, IndexException {
         int fieldCount = ctx.getFieldCount();
         int numTuples = checkTuples.size();
         ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
         // Perform bulk load.
-        IIndexBulkLoadContext bulkLoadCtx = ctx.getIndex().beginBulkLoad(0.7f);
+        IIndexBulkLoader bulkLoader = ctx.getIndex().createBulkLoader(0.7f, false, numTuples);
         int c = 1;
         for (CheckTuple checkTuple : checkTuples) {
             if (LOGGER.isLoggable(Level.INFO)) {
@@ -237,14 +252,14 @@
                 }
             }
             createTupleFromCheckTuple(checkTuple, tupleBuilder, tuple, ctx.getFieldSerdes());
-            ctx.getIndex().bulkLoadAddTuple(tuple, bulkLoadCtx);
+            bulkLoader.add(tuple);
             c++;
         }
-        ctx.getIndex().endBulkLoad(bulkLoadCtx);
+        bulkLoader.end();
     }
 
     @SuppressWarnings("unchecked")
-    public void deleteTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void deleteTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         ArrayTupleBuilder deleteTupleBuilder = createDeleteTupleBuilder(ctx);
         ArrayTupleReference deleteTuple = new ArrayTupleReference();
         int numCheckTuples = ctx.getCheckTuples().size();
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
new file mode 100644
index 0000000..f962200
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
@@ -0,0 +1,182 @@
+/*
+ * Copyright 2009-2010 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.config;
+
+/**
+ * Global parameters used for executing access method JUnit tests.
+ */
+public class AccessMethodTestsConfig {
+    // Test params for RTree, LSMRTree and LSMRTreeWithAntiMatterTuples.
+    public static final int RTREE_NUM_TUPLES_TO_INSERT = 100;
+    public static final int RTREE_NUM_INSERT_ROUNDS = 2;
+    public static final int RTREE_NUM_DELETE_ROUNDS = 2;
+    public static final int RTREE_MULTITHREAD_NUM_OPERATIONS = 200;
+    public static final boolean RTREE_TEST_RSTAR_POLICY = true;
+    // Test params for LSMRTree and LSMRTreeWithAntiMatterTuples.
+    public static final int LSM_RTREE_BULKLOAD_ROUNDS = 5;
+    public static final int LSM_RTREE_MAX_TREES_TO_MERGE = 3;
+    public static final boolean LSM_RTREE_TEST_RSTAR_POLICY = false;
+
+    // Test params for BTree, LSMBTree.
+    public static final int BTREE_NUM_TUPLES_TO_INSERT = 100;
+    public static final int BTREE_NUM_INSERT_ROUNDS = 3;
+    public static final int BTREE_NUM_DELETE_ROUNDS = 3;
+    public static final int BTREE_NUM_UPDATE_ROUNDS = 3;
+    public static final int BTREE_MULTITHREAD_NUM_OPERATIONS = 200;
+    // Test params for LSMBTree only.
+    public static final int LSM_BTREE_BULKLOAD_ROUNDS = 5;
+    public static final int LSM_BTREE_MAX_TREES_TO_MERGE = 10;
+
+    // Mem configuration for RTree.
+    public static final int RTREE_PAGE_SIZE = 512;
+    public static final int RTREE_NUM_PAGES = 1000;
+    public static final int RTREE_MAX_OPEN_FILES = 10;
+    public static final int RTREE_HYRACKS_FRAME_SIZE = 128;
+
+    // Mem configuration for LSMRTree and LSMRTreeWithAntiMatterTuples.
+    public static final int LSM_RTREE_DISK_PAGE_SIZE = 512;
+    public static final int LSM_RTREE_DISK_NUM_PAGES = 1000;
+    public static final int LSM_RTREE_DISK_MAX_OPEN_FILES = 2000;
+    public static final int LSM_RTREE_MEM_PAGE_SIZE = 512;
+    public static final int LSM_RTREE_MEM_NUM_PAGES = 1000;
+    public static final int LSM_RTREE_HYRACKS_FRAME_SIZE = 128;
+
+    // Mem configuration for BTree.
+    public static final int BTREE_PAGE_SIZE = 256;
+    public static final int BTREE_NUM_PAGES = 100;
+    public static final int BTREE_MAX_OPEN_FILES = 10;
+    public static final int BTREE_HYRACKS_FRAME_SIZE = 128;
+
+    // Mem configuration for LSMBTree.
+    public static final int LSM_BTREE_DISK_PAGE_SIZE = 256;
+    public static final int LSM_BTREE_DISK_NUM_PAGES = 1000;
+    public static final int LSM_BTREE_DISK_MAX_OPEN_FILES = 200;
+    public static final int LSM_BTREE_MEM_PAGE_SIZE = 256;
+    public static final int LSM_BTREE_MEM_NUM_PAGES = 100;
+    public static final int LSM_BTREE_HYRACKS_FRAME_SIZE = 128;
+
+    // Mem configuration for Inverted Index.
+    public static final int LSM_INVINDEX_DISK_PAGE_SIZE = 1024;
+    public static final int LSM_INVINDEX_DISK_NUM_PAGES = 1000;
+    public static final int LSM_INVINDEX_DISK_MAX_OPEN_FILES = 1000;
+    public static final int LSM_INVINDEX_MEM_PAGE_SIZE = 1024;
+    public static final int LSM_INVINDEX_MEM_NUM_PAGES = 100;
+    public static final int LSM_INVINDEX_HYRACKS_FRAME_SIZE = 32768;
+    // Test parameters.
+    public static final int LSM_INVINDEX_NUM_DOCS_TO_INSERT = 100;
+    // Used for full-fledged search test.
+    public static final int LSM_INVINDEX_NUM_DOC_QUERIES = 1000;
+    public static final int LSM_INVINDEX_NUM_RANDOM_QUERIES = 1000;
+    // Used for non-search tests to sanity check index searches.
+    public static final int LSM_INVINDEX_TINY_NUM_DOC_QUERIES = 200;
+    public static final int LSM_INVINDEX_TINY_NUM_RANDOM_QUERIES = 200;
+    public static final int LSM_INVINDEX_NUM_BULKLOAD_ROUNDS = 5;
+    public static final int LSM_INVINDEX_MAX_TREES_TO_MERGE = 5;
+    public static final int LSM_INVINDEX_NUM_INSERT_ROUNDS = 3;
+    public static final int LSM_INVINDEX_NUM_DELETE_ROUNDS = 3;
+    // Allocate a generous size to make sure we have enough elements for all tests.
+    public static final int LSM_INVINDEX_SCAN_COUNT_ARRAY_SIZE = 1000000;
+    public static final int LSM_INVINDEX_MULTITHREAD_NUM_OPERATIONS = 200;
+
+    // Test params for BloomFilter
+    public static final int BLOOM_FILTER_NUM_TUPLES_TO_INSERT = 100;
+
+    // Mem configuration for BloomFilter.
+    public static final int BLOOM_FILTER_PAGE_SIZE = 256;
+    public static final int BLOOM_FILTER_NUM_PAGES = 1000;
+    public static final int BLOOM_FILTER_MAX_OPEN_FILES = 10;
+    public static final int BLOOM_FILTER_HYRACKS_FRAME_SIZE = 128;
+
+}
+
+/* ORIGINAL TEST PARAMETERS: DO NOT EDIT!
+// Test params for RTree, LSMRTree and LSMRTreeWithAntiMatterTuples.
+public static final int RTREE_NUM_TUPLES_TO_INSERT = 10000;
+public static final int RTREE_NUM_INSERT_ROUNDS = 2;
+public static final int RTREE_NUM_DELETE_ROUNDS = 2;
+public static final int RTREE_MULTITHREAD_NUM_OPERATIONS = 10000;
+// Test params for LSMRTree and LSMRTreeWithAntiMatterTuples.
+public static final int LSM_RTREE_BULKLOAD_ROUNDS = 5;
+public static final int LSM_RTREE_MAX_TREES_TO_MERGE = 3;	
+
+// Test params for BTree, LSMBTree.
+public static final int BTREE_NUM_TUPLES_TO_INSERT = 10000;
+public static final int BTREE_NUM_INSERT_ROUNDS = 3;
+public static final int BTREE_NUM_DELETE_ROUNDS = 3;
+public static final int BTREE_NUM_UPDATE_ROUNDS = 3;
+public static final int BTREE_MULTITHREAD_NUM_OPERATIONS = 10000;
+// Test params for LSMBTree only.
+public static final int LSM_BTREE_BULKLOAD_ROUNDS = 5;
+public static final int LSM_BTREE_MAX_TREES_TO_MERGE = 10;
+	
+	
+// Mem configuration for RTree.
+public static final int RTREE_PAGE_SIZE = 512;
+public static final int RTREE_NUM_PAGES = 1000;
+public static final int RTREE_MAX_OPEN_FILES = 10;
+public static final int RTREE_HYRACKS_FRAME_SIZE = 128;
+	
+// Mem configuration for LSMRTree and LSMRTreeWithAntiMatterTuples.
+public static final int LSM_RTREE_DISK_PAGE_SIZE = 256;
+public static final int LSM_RTREE_DISK_NUM_PAGES = 1000;
+public static final int LSM_RTREE_DISK_MAX_OPEN_FILES = 2000;
+public static final int LSM_RTREE_MEM_PAGE_SIZE = 256;
+public static final int LSM_RTREE_MEM_NUM_PAGES = 1000;
+public static final int LSM_RTREE_HYRACKS_FRAME_SIZE = 128;
+	
+// Mem configuration for BTree.
+public static final int BTREE_PAGE_SIZE = 256;
+public static final int BTREE_NUM_PAGES = 100;
+public static final int BTREE_MAX_OPEN_FILES = 10;
+public static final int BTREE_HYRACKS_FRAME_SIZE = 128;
+	
+// Mem configuration for LSMBTree.
+public static final int LSM_BTREE_DISK_PAGE_SIZE = 256;
+public static final int LSM_BTREE_DISK_NUM_PAGES = 1000;
+public static final int LSM_BTREE_DISK_MAX_OPEN_FILES = 200;
+public static final int LSM_BTREE_MEM_PAGE_SIZE = 256;
+public static final int LSM_BTREE_MEM_NUM_PAGES = 100;
+public static final int LSM_BTREE_HYRACKS_FRAME_SIZE = 128;
+
+// Mem configuration for Inverted Index.
+public static final int INVINDEX_PAGE_SIZE = 32768;
+public static final int INVINDEX_NUM_PAGES = 100;
+public static final int INVINDEX_MAX_OPEN_FILES = 10;
+public static final int INVINDEX_HYRACKS_FRAME_SIZE = 32768;
+
+// Mem configuration for Inverted Index.
+public static final int LSM_INVINDEX_DISK_PAGE_SIZE = 1024;
+public static final int LSM_INVINDEX_DISK_NUM_PAGES = 1000;
+public static final int LSM_INVINDEX_DISK_MAX_OPEN_FILES = 1000;
+public static final int LSM_INVINDEX_MEM_PAGE_SIZE = 1024;
+public static final int LSM_INVINDEX_MEM_NUM_PAGES = 100;
+public static final int LSM_INVINDEX_HYRACKS_FRAME_SIZE = 32768;
+// Test parameters.
+public static final int LSM_INVINDEX_NUM_DOCS_TO_INSERT = 10000;
+// Used for full-fledged search test.
+public static final int LSM_INVINDEX_NUM_DOC_QUERIES = 1000;
+public static final int LSM_INVINDEX_NUM_RANDOM_QUERIES = 1000;
+// Used for non-search tests to sanity check index searches.
+public static final int LSM_INVINDEX_TINY_NUM_DOC_QUERIES = 200;
+public static final int LSM_INVINDEX_TINY_NUM_RANDOM_QUERIES = 200;
+public static final int LSM_INVINDEX_NUM_BULKLOAD_ROUNDS = 5;
+public static final int LSM_INVINDEX_MAX_TREES_TO_MERGE = 5;
+public static final int LSM_INVINDEX_NUM_INSERT_ROUNDS = 3;
+public static final int LSM_INVINDEX_NUM_DELETE_ROUNDS = 3;
+// Allocate a generous size to make sure we have enough elements for all tests.
+public static final int LSM_INVINDEX_SCAN_COUNT_ARRAY_SIZE = 1000000;
+public static final int LSM_INVINDEX_MULTITHREAD_NUM_OPERATIONS = 10000;
+*/
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 198ac58..54ad1fe 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
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeBulkLoadTest extends AbstractRTreeTestDriver {
@@ -27,15 +28,19 @@
     private final RTreeTestUtils rTreeTestUtils;
     private final int bulkLoadRounds;
 
-    public AbstractRTreeBulkLoadTest(int bulkLoadRounds) {
+    public AbstractRTreeBulkLoadTest(int bulkLoadRounds, boolean testRstarPolicy) {
+        super(testRstarPolicy);
         this.bulkLoadRounds = bulkLoadRounds;
         this.rTreeTestUtils = new RTreeTestUtils();
     }
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key) throws Exception {
-        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys);
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key,
+            RTreePolicyType rtreePolicyType) throws Exception {
+        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.
@@ -49,7 +54,8 @@
             rTreeTestUtils.checkDiskOrderScan(ctx);
             rTreeTestUtils.checkRangeSearch(ctx, key);
         }
-        ctx.getIndex().close();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeDeleteTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeDeleteTest.java
index e70f433..18d042b 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeDeleteTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeDeleteTest.java
@@ -20,23 +20,29 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 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.rtree.frames.RTreePolicyType;
 
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeDeleteTest extends AbstractRTreeTestDriver {
 
     private final RTreeTestUtils rTreeTestUtils;
 
-    private static final int numInsertRounds = 2;
-    private static final int numDeleteRounds = 2;
+    private static final int numInsertRounds = AccessMethodTestsConfig.RTREE_NUM_INSERT_ROUNDS;
+    private static final int numDeleteRounds = AccessMethodTestsConfig.RTREE_NUM_DELETE_ROUNDS;
 
-    public AbstractRTreeDeleteTest() {
+    public AbstractRTreeDeleteTest(boolean testRstarPolicy) {
+    	super(testRstarPolicy);
         this.rTreeTestUtils = new RTreeTestUtils();
     }
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key) throws Exception {
-        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys);
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key,
+            RTreePolicyType rtreePolicyType) throws Exception {
+        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         for (int i = 0; i < numInsertRounds; i++) {
             // We assume all fieldSerdes are of the same type. Check the first
             // one to determine which field types to generate.
@@ -54,7 +60,8 @@
                 rTreeTestUtils.checkRangeSearch(ctx, key);
             }
         }
-        ctx.getIndex().close();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
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 7192c53..f93e9b6 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
@@ -27,21 +27,26 @@
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 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.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
 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.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
@@ -52,16 +57,14 @@
 
     protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories) throws TreeIndexException;
-
-    protected abstract int getIndexFileId();
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType)
+            throws TreeIndexException;
 
     /**
-     * Two Dimensions Example.
-     * 
-     * Create an RTree index of two dimensions, where they keys are of type
-     * integer, and the payload is two integer values. Fill index with random
-     * values using insertions (not bulk load). Perform scans and range search.
+     * Two Dimensions Example. Create an RTree index of two dimensions, where
+     * they keys are of type integer, and the payload is two integer values.
+     * Fill index with random values using insertions (not bulk load). Perform
+     * scans and range search.
      */
     @Test
     public void twoDimensionsExample() throws Exception {
@@ -106,10 +109,10 @@
         IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
                 rtreeCmpFactories.length, IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -117,7 +120,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
         int numInserts = 10000;
         for (int i = 0; i < numInserts; i++) {
             int p1x = rnd.nextInt();
@@ -156,15 +160,245 @@
 
         rangeSearch(rtreeCmpFactories, indexAccessor, fieldSerdes, key);
 
-        treeIndex.close();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Two Dimensions Example.
-     * 
-     * Create an RTree index of three dimensions, where they keys are of type
-     * double, and the payload is one double value. Fill index with random
-     * values using insertions (not bulk load). Perform scans and range search.
+     * This test the rtree page split. Originally this test didn't pass since
+     * the rtree assumes always that there will be enough space for the new
+     * tuple after split. Now it passes since if there is not space in the
+     * designated page, then we will just insert it in the other split page.
+     */
+    @Test
+    public void rTreePageSplitTestExample() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RTree page split test.");
+        }
+
+        // Declare fields.
+        int fieldCount = 5;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[2] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[3] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        // Declare field serdes.
+        ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+
+        // Declare RTree keys.
+        int rtreeKeyFieldCount = 4;
+        IBinaryComparatorFactory[] rtreeCmpFactories = new IBinaryComparatorFactory[rtreeKeyFieldCount];
+        rtreeCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[3] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
+        // Declare BTree keys, this will only be used for LSMRTree
+        int btreeKeyFieldCount = 5;
+        IBinaryComparatorFactory[] btreeCmpFactories = new IBinaryComparatorFactory[btreeKeyFieldCount];
+        btreeCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[3] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[4] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
+        // create value providers
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                rtreeCmpFactories.length, IntegerPointable.FACTORY);
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+
+        treeIndex.create();
+        treeIndex.activate();
+
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
+
+        int p1x = rnd.nextInt();
+        int p1y = rnd.nextInt();
+        int p2x = rnd.nextInt();
+        int p2y = rnd.nextInt();
+        String data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        treeIndex.deactivate();
+        treeIndex.destroy();
+    }
+
+    /**
+     * This test the r*tree page split. Originally this test didn't pass since
+     * the r*tree assumes always that there will be enough space for the new
+     * tuple after split. Now it passes since if there is not space in the
+     * designated page, then we will just insert it in the other split page.
+     */
+    @Test
+    public void rStarTreePageSplitTestExample() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("R*Tree page split test.");
+        }
+
+        // Declare fields.
+        int fieldCount = 5;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[2] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[3] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
+        // Declare field serdes.
+        ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+
+        // Declare RTree keys.
+        int rtreeKeyFieldCount = 4;
+        IBinaryComparatorFactory[] rtreeCmpFactories = new IBinaryComparatorFactory[rtreeKeyFieldCount];
+        rtreeCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[3] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
+        // Declare BTree keys, this will only be used for LSMRTree
+        int btreeKeyFieldCount = 5;
+        IBinaryComparatorFactory[] btreeCmpFactories = new IBinaryComparatorFactory[btreeKeyFieldCount];
+        btreeCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[3] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[4] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
+        // create value providers
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                rtreeCmpFactories.length, IntegerPointable.FACTORY);
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RSTARTREE);
+
+        treeIndex.create();
+        treeIndex.activate();
+
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
+
+        int p1x = rnd.nextInt();
+        int p1y = rnd.nextInt();
+        int p2x = rnd.nextInt();
+        int p2y = rnd.nextInt();
+        String data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        treeIndex.deactivate();
+        treeIndex.destroy();
+    }
+
+    /**
+     * Two Dimensions Example. Create an RTree index of three dimensions, where
+     * they keys are of type double, and the payload is one double value. Fill
+     * index with random values using insertions (not bulk load). Perform scans
+     * and range search.
      */
     @Test
     public void threeDimensionsExample() throws Exception {
@@ -213,10 +447,10 @@
         IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
                 rtreeCmpFactories.length, DoublePointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -224,7 +458,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
         int numInserts = 10000;
         for (int i = 0; i < numInserts; i++) {
             double p1x = rnd.nextDouble();
@@ -265,16 +500,15 @@
 
         rangeSearch(rtreeCmpFactories, indexAccessor, fieldSerdes, key);
 
-        treeIndex.close();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Deletion Example.
-     * 
-     * Create an RTree index of two dimensions, where they keys are of type
-     * integer, and the payload is one integer value. Fill index with random
-     * values using insertions, then delete entries one-by-one. Repeat procedure
-     * a few times on same RTree.
+     * Deletion Example. Create an RTree index of two dimensions, where they
+     * keys are of type integer, and the payload is one integer value. Fill
+     * index with random values using insertions, then delete entries
+     * one-by-one. Repeat procedure a few times on same RTree.
      */
     @Test
     public void deleteExample() throws Exception {
@@ -312,14 +546,15 @@
         IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
                 rtreeCmpFactories.length, IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+        treeIndex.create();
+        treeIndex.activate();
 
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
 
         int runs = 3;
         for (int run = 0; run < runs; run++) {
@@ -395,14 +630,12 @@
                 break;
             }
         }
-        treeIndex.close();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Bulk load example.
-     * 
-     * Load a tree with 10,000 tuples.
-     * 
+     * Bulk load example. Load a tree with 10,000 tuples.
      */
     @Test
     public void bulkLoadExample() throws Exception {
@@ -444,10 +677,10 @@
         IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
                 rtreeCmpFactories.length, IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+        treeIndex.create();
+        treeIndex.activate();
 
         // Load records.
         int numInserts = 10000;
@@ -455,7 +688,7 @@
             LOGGER.info("Bulk loading " + numInserts + " tuples");
         }
         long start = System.currentTimeMillis();
-        IIndexBulkLoadContext bulkLoadCtx = treeIndex.beginBulkLoad(0.7f);
+        IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, numInserts);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
@@ -469,16 +702,17 @@
 
             TupleUtils.createIntegerTuple(tb, tuple, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                     Math.max(p1y, p2y), pk);
-            treeIndex.bulkLoadAddTuple(tuple, bulkLoadCtx);
+            bulkLoader.add(tuple);
         }
 
-        treeIndex.endBulkLoad(bulkLoadCtx);
+        bulkLoader.end();
         long end = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(numInserts + " tuples loaded in " + (end - start) + "ms");
         }
 
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
 
         // Build key.
         ArrayTupleBuilder keyTb = new ArrayTupleBuilder(rtreeKeyFieldCount);
@@ -487,7 +721,8 @@
 
         rangeSearch(rtreeCmpFactories, indexAccessor, fieldSerdes, key);
 
-        treeIndex.close();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     private void scan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes) throws Exception {
@@ -511,14 +746,13 @@
         }
     }
 
-    private void diskOrderScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes)
-            throws Exception {
+    private void diskOrderScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes) throws Exception {
         try {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Disk-Order Scan:");
             }
             ITreeIndexAccessor treeIndexAccessor = (ITreeIndexAccessor) indexAccessor;
-            TreeDiskOrderScanCursor diskOrderCursor = (TreeDiskOrderScanCursor) treeIndexAccessor
+            TreeIndexDiskOrderScanCursor diskOrderCursor = (TreeIndexDiskOrderScanCursor) treeIndexAccessor
                     .createDiskOrderScanCursor();
             treeIndexAccessor.diskOrderScan(diskOrderCursor);
             try {
@@ -540,12 +774,12 @@
                 LOGGER.info("Ignoring disk-order scan since it's not supported.");
             }
         } catch (ClassCastException e) {
-			// Ignore exception because IIndexAccessor sometimes isn't
-			// an ITreeIndexAccessor, e.g., for the LSMRTree.
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Ignoring disk-order scan since it's not supported.");
-			}
-		}
+            // Ignore exception because IIndexAccessor sometimes isn't
+            // an ITreeIndexAccessor, e.g., for the LSMRTree.
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring disk-order scan since it's not supported.");
+            }
+        }
     }
 
     private void rangeSearch(IBinaryComparatorFactory[] cmpFactories, IIndexAccessor indexAccessor,
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeInsertTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeInsertTest.java
index cdd6ee0..eb90989 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeInsertTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeInsertTest.java
@@ -20,29 +20,32 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 /**
  * Tests the RTree insert operation with integer and double fields using various
  * numbers of dimensions and payload fields.
- * 
  * Each tests first fills an RTree with randomly generated tuples. We compare
  * the following operations against expected results: 1. RTree scan. 3.
  * Disk-order scan. 4. Range search.
- * 
  */
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeInsertTest extends AbstractRTreeTestDriver {
 
     private final RTreeTestUtils rTreeTestUtils;
 
-    public AbstractRTreeInsertTest() {
+    public AbstractRTreeInsertTest(boolean testRstarPolicy) {
+    	super(testRstarPolicy);
         this.rTreeTestUtils = new RTreeTestUtils();
     }
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key) throws Exception {
-        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys);
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key,
+            RTreePolicyType rtreePolicyType) throws Exception {
+        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         // 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) {
@@ -54,7 +57,8 @@
         rTreeTestUtils.checkScan(ctx);
         rTreeTestUtils.checkDiskOrderScan(ctx);
         rTreeTestUtils.checkRangeSearch(ctx, key);
-        ctx.getIndex().close();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java
index 2c185a5..0c1b9a5 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java
@@ -31,25 +31,32 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestWorkerFactory;
-import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.IndexMultiThreadTestDriver;
 import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
-import edu.uci.ics.hyracks.storage.am.common.TreeIndexMultiThreadTestDriver;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeMultiThreadTest {
 
+    protected final boolean testRstarPolicy;
+
+    public AbstractRTreeMultiThreadTest(boolean testRstarPolicy) {
+        this.testRstarPolicy = testRstarPolicy;
+    }
+
     protected final Logger LOGGER = Logger.getLogger(AbstractRTreeMultiThreadTest.class.getName());
 
     // Machine-specific number of threads to use for testing.
     protected final int REGULAR_NUM_THREADS = Runtime.getRuntime().availableProcessors();
     // Excessive number of threads for testing.
     protected final int EXCESSIVE_NUM_THREADS = Runtime.getRuntime().availableProcessors() * 4;
-    protected final int NUM_OPERATIONS = 5000;
+    protected final int NUM_OPERATIONS = AccessMethodTestsConfig.RTREE_MULTITHREAD_NUM_OPERATIONS;
 
     protected ArrayList<TestWorkloadConf> workloadConfs = getTestWorkloadConf();
 
@@ -59,27 +66,19 @@
 
     protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories) throws TreeIndexException;
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType)
+            throws TreeIndexException;
 
-    protected abstract int getFileId();
-
-    protected abstract ITreeIndexTestWorkerFactory getWorkerFactory();
+    protected abstract IIndexTestWorkerFactory getWorkerFactory();
 
     protected abstract ArrayList<TestWorkloadConf> getTestWorkloadConf();
 
     protected abstract String getIndexTypeName();
 
-    protected static float[] getUniformOpProbs(TestOperation[] ops) {
-        float[] opProbs = new float[ops.length];
-        for (int i = 0; i < ops.length; i++) {
-            opProbs[i] = 1.0f / (float) ops.length;
-        }
-        return opProbs;
-    }
-
     protected void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, int numThreads,
-            TestWorkloadConf conf, String dataMsg) throws HyracksException, InterruptedException, TreeIndexException {
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType,
+            int numThreads, TestWorkloadConf conf, String dataMsg) throws HyracksException, InterruptedException,
+            TreeIndexException {
         setUp();
 
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -93,15 +92,16 @@
         IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
                 fieldSerdes.length);
 
-        ITreeIndex index = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        ITreeIndexTestWorkerFactory workerFactory = getWorkerFactory();
+        ITreeIndex index = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
+                rtreePolicyType);
+        IIndexTestWorkerFactory workerFactory = getWorkerFactory();
 
         // 4 batches per thread.
         int batchSize = (NUM_OPERATIONS / numThreads) / 4;
 
-        TreeIndexMultiThreadTestDriver driver = new TreeIndexMultiThreadTestDriver(index, workerFactory, fieldSerdes,
-                conf.ops, conf.opProbs);
-        driver.init(getFileId());
+        IndexMultiThreadTestDriver driver = new IndexMultiThreadTestDriver(index, workerFactory, fieldSerdes, conf.ops,
+                conf.opProbs);
+        driver.init();
         long[] times = driver.run(numThreads, 1, NUM_OPERATIONS, batchSize);
         driver.deinit();
 
@@ -113,7 +113,7 @@
     }
 
     @Test
-    public void twoDimensionsInt() throws InterruptedException, HyracksException, TreeIndexException {
+    public void rtreeTwoDimensionsInt() throws InterruptedException, HyracksException, TreeIndexException {
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
@@ -125,13 +125,36 @@
         String dataMsg = "Two Dimensions Of Integer Values";
 
         for (TestWorkloadConf conf : workloadConfs) {
-            runTest(fieldSerdes, valueProviderFactories, numKeys, REGULAR_NUM_THREADS, conf, dataMsg);
-            runTest(fieldSerdes, valueProviderFactories, numKeys, EXCESSIVE_NUM_THREADS, conf, dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, EXCESSIVE_NUM_THREADS, conf,
+                    dataMsg);
         }
     }
 
     @Test
-    public void fourDimensionsDouble() throws InterruptedException, HyracksException, TreeIndexException {
+    public void rtreeTwoDimensionsDouble() throws Exception {
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+
+        String dataMsg = "Two Dimensions Of Double Values";
+
+        for (TestWorkloadConf conf : workloadConfs) {
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, EXCESSIVE_NUM_THREADS, conf,
+                    dataMsg);
+        }
+
+    }
+
+    @Test
+    public void rtreeFourDimensionsDouble() throws InterruptedException, HyracksException, TreeIndexException {
         ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
                 DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
                 DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
@@ -145,8 +168,95 @@
         String dataMsg = "Four Dimensions Of Double Values";
 
         for (TestWorkloadConf conf : workloadConfs) {
-            runTest(fieldSerdes, valueProviderFactories, numKeys, REGULAR_NUM_THREADS, conf, dataMsg);
-            runTest(fieldSerdes, valueProviderFactories, numKeys, EXCESSIVE_NUM_THREADS, conf, dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, EXCESSIVE_NUM_THREADS, conf,
+                    dataMsg);
         }
     }
+
+    @Test
+    public void rstartreeTwoDimensionsInt() throws InterruptedException, HyracksException, TreeIndexException {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree Multithread Test With Two Dimensions With Integer Keys.");
+            }
+            return;
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, IntegerPointable.FACTORY);
+
+        String dataMsg = "Two Dimensions Of Integer Values";
+
+        for (TestWorkloadConf conf : workloadConfs) {
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, EXCESSIVE_NUM_THREADS,
+                    conf, dataMsg);
+        }
+    }
+
+    @Test
+    public void rstartreeTwoDimensionsDouble() throws Exception {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree Multithread Test With Two Dimensions With Double Keys.");
+            }
+            return;
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+
+        String dataMsg = "Two Dimensions Of Double Values";
+
+        for (TestWorkloadConf conf : workloadConfs) {
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, EXCESSIVE_NUM_THREADS,
+                    conf, dataMsg);
+        }
+
+    }
+
+    @Test
+    public void rstartreeFourDimensionsDouble() throws InterruptedException, HyracksException, TreeIndexException {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree Multithread Test With Four Dimensions With Double Keys.");
+            }
+            return;
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 8;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+
+        String dataMsg = "Four Dimensions Of Double Values";
+
+        for (TestWorkloadConf conf : workloadConfs) {
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, EXCESSIVE_NUM_THREADS,
+                    conf, dataMsg);
+        }
+    }
+
 }
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 9affc47..eed8df6 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
@@ -15,17 +15,17 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree;
 
-import java.util.ArrayList;
 import java.util.Collection;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestContext;
+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;
 
 @SuppressWarnings("rawtypes")
-public abstract class AbstractRTreeTestContext extends TreeIndexTestContext<RTreeCheckTuple> {
-    private final ArrayList<RTreeCheckTuple> checkTuples = new ArrayList<RTreeCheckTuple>();
-
+public abstract class AbstractRTreeTestContext extends IndexTestContext<RTreeCheckTuple> {
+    private final HashMultiSet<RTreeCheckTuple> checkTuples = new HashMultiSet<RTreeCheckTuple>();
+	
     public AbstractRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
         super(fieldSerdes, treeIndex);
     }
@@ -34,5 +34,4 @@
     public Collection<RTreeCheckTuple> getCheckTuples() {
         return checkTuples;
     }
-
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java
index 10f4364..de92823 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java
@@ -29,26 +29,36 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 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.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeTestDriver {
+    protected final boolean testRstarPolicy;
+
+    public AbstractRTreeTestDriver(boolean testRstarPolicy) {
+        this.testRstarPolicy = testRstarPolicy;
+    }
+
     protected final Logger LOGGER = Logger.getLogger(AbstractRTreeTestDriver.class.getName());
 
-    protected static final int numTuplesToInsert = 5000;
+    protected static final int numTuplesToInsert = AccessMethodTestsConfig.RTREE_NUM_TUPLES_TO_INSERT;
 
     protected abstract AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys) throws Exception;
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
+            throws Exception;
 
     protected abstract Random getRandom();
 
     protected abstract void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key) throws Exception;
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key,
+            RTreePolicyType rtreePolicyType) throws Exception;
 
     protected abstract String getTestOpName();
 
     @Test
-    public void twoDimensionsInt() throws Exception {
+    public void rtreeTwoDimensionsInt() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("RTree " + getTestOpName() + " Test With Two Dimensions With Integer Keys.");
         }
@@ -64,12 +74,12 @@
         // and the top right coordinates are 1000, 1000
         ITupleReference key = TupleUtils.createIntegerTuple(-1000, -1000, 1000, 1000);
 
-        runTest(fieldSerdes, valueProviderFactories, numKeys, key);
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RTREE);
 
     }
 
     @Test
-    public void twoDimensionsDouble() throws Exception {
+    public void rtreeTwoDimensionsDouble() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("RTree " + getTestOpName() + " Test With Two Dimensions With Double Keys.");
         }
@@ -85,12 +95,12 @@
         // -1000.0 and the top right coordinates are 1000.0, 1000.0
         ITupleReference key = TupleUtils.createDoubleTuple(-1000.0, -1000.0, 1000.0, 1000.0);
 
-        runTest(fieldSerdes, valueProviderFactories, numKeys, key);
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RTREE);
 
     }
 
     @Test
-    public void fourDimensionsDouble() throws Exception {
+    public void rtreeFourDimensionsDouble() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("RTree " + getTestOpName() + " Test With Four Dimensions With Double Keys.");
         }
@@ -110,7 +120,90 @@
         ITupleReference key = TupleUtils.createDoubleTuple(-1000.0, -1000.0, -1000.0, -1000.0, 1000.0, 1000.0, 1000.0,
                 1000.0);
 
-        runTest(fieldSerdes, valueProviderFactories, numKeys, key);
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RTREE);
+    }
 
+    @Test
+    public void rstartreeTwoDimensionsInt() throws Exception {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree " + getTestOpName() + " Test With Two Dimensions With Integer Keys.");
+            }
+            return;
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RTree " + getTestOpName() + " Test With Two Dimensions With Integer Keys.");
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, IntegerPointable.FACTORY);
+        // Range search, the rectangle bottom left coordinates are -1000, -1000
+        // and the top right coordinates are 1000, 1000
+        ITupleReference key = TupleUtils.createIntegerTuple(-1000, -1000, 1000, 1000);
+
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RSTARTREE);
+
+    }
+
+    @Test
+    public void rstartreeTwoDimensionsDouble() throws Exception {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree " + getTestOpName() + " Test With Two Dimensions With Double Keys.");
+            }
+            return;
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RTree " + getTestOpName() + " Test With Two Dimensions With Double Keys.");
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+        // Range search, the rectangle bottom left coordinates are -1000.0,
+        // -1000.0 and the top right coordinates are 1000.0, 1000.0
+        ITupleReference key = TupleUtils.createDoubleTuple(-1000.0, -1000.0, 1000.0, 1000.0);
+
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RSTARTREE);
+
+    }
+
+    @Test
+    public void rstartreeFourDimensionsDouble() throws Exception {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree " + getTestOpName() + " Test With Four Dimensions With Double Keys.");
+            }
+            return;
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RTree " + getTestOpName() + " Test With Four Dimensions With Double Keys.");
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 8;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+        // Range search, the rectangle bottom left coordinates are -1000.0,
+        // -1000.0, -1000.0, -1000.0 and the top right coordinates are 1000.0,
+        // 1000.0, 1000.0, 1000.0
+        ITupleReference key = TupleUtils.createDoubleTuple(-1000.0, -1000.0, -1000.0, -1000.0, 1000.0, 1000.0, 1000.0,
+                1000.0);
+
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RSTARTREE);
     }
 }
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeCheckTuple.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeCheckTuple.java
index 98800e5..c498136 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeCheckTuple.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeCheckTuple.java
@@ -27,8 +27,8 @@
     @Override
     public boolean equals(Object o) {
         RTreeCheckTuple<T> other = (RTreeCheckTuple<T>) o;
-        for (int i = 0; i < tuple.length; i++) {
-            int cmp = tuple[i].compareTo(other.get(i));
+        for (int i = 0; i < fields.length; i++) {
+            int cmp = fields[i].compareTo(other.getField(i));
             if (cmp != 0) {
                 return false;
             }
@@ -41,11 +41,11 @@
         int maxFieldPos = numKeys / 2;
         for (int i = 0; i < maxFieldPos; i++) {
             int j = maxFieldPos + i;
-            int cmp = tuple[i].compareTo(other.get(j));
+            int cmp = fields[i].compareTo(other.getField(j));
             if (cmp > 0) {
                 return false;
             }
-            cmp = tuple[j].compareTo(other.get(i));
+            cmp = fields[j].compareTo(other.getField(i));
             if (cmp < 0) {
                 return false;
             }
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java
index a23f375..067c6cb 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java
@@ -15,12 +15,13 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestContext;
 import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.util.HashMultiSet;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
@@ -32,9 +33,9 @@
 
     @SuppressWarnings("unchecked")
     // Create a new ArrayList containing the elements satisfying the search key
-    public ArrayList<RTreeCheckTuple> getRangeSearchExpectedResults(ArrayList<RTreeCheckTuple> checkTuples,
+    public HashMultiSet<RTreeCheckTuple> getRangeSearchExpectedResults(Collection<RTreeCheckTuple> checkTuples,
             RTreeCheckTuple key) {
-        ArrayList<RTreeCheckTuple> expectedResult = new ArrayList<RTreeCheckTuple>();
+        HashMultiSet<RTreeCheckTuple> expectedResult = new HashMultiSet<RTreeCheckTuple>();
         Iterator<RTreeCheckTuple> iter = checkTuples.iterator();
         while (iter.hasNext()) {
             RTreeCheckTuple t = iter.next();
@@ -45,7 +46,7 @@
         return expectedResult;
     }
 
-    public void checkRangeSearch(ITreeIndexTestContext ictx, ITupleReference key) throws Exception {
+    public void checkRangeSearch(IIndexTestContext ictx, ITupleReference key) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Testing Range Search.");
         }
@@ -61,14 +62,14 @@
         RTreeCheckTuple keyCheck = (RTreeCheckTuple) createCheckTupleFromTuple(key, ctx.getFieldSerdes(),
                 cmp.getKeyFieldCount());
 
-        ArrayList<RTreeCheckTuple> expectedResult = null;
+        HashMultiSet<RTreeCheckTuple> expectedResult = null;
 
-        expectedResult = getRangeSearchExpectedResults((ArrayList<RTreeCheckTuple>) ctx.getCheckTuples(), keyCheck);
+        expectedResult = getRangeSearchExpectedResults(ctx.getCheckTuples(), keyCheck);
         checkExpectedResults(searchCursor, expectedResult, ctx.getFieldSerdes(), ctx.getKeyFieldCount(), null);
     }
 
     @SuppressWarnings("unchecked")
-    public void insertDoubleTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void insertDoubleTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         double[] fieldValues = new double[ctx.getFieldCount()];
@@ -111,7 +112,7 @@
             fieldValues[k] = secondValue;
         }
     }
-    
+
     private void setDoublePayloadFields(double[] fieldValues, int numKeyFields, int numFields) {
         for (int j = numKeyFields; j < numFields; j++) {
             fieldValues[j] = doublePayloadValue++;
@@ -122,13 +123,13 @@
     protected CheckTuple createDoubleCheckTuple(double[] fieldValues, int numKeyFields) {
         RTreeCheckTuple<Double> checkTuple = new RTreeCheckTuple<Double>(fieldValues.length, numKeyFields);
         for (double v : fieldValues) {
-            checkTuple.add(v);
+            checkTuple.appendField(v);
         }
         return checkTuple;
     }
 
     @SuppressWarnings("unchecked")
-    public void bulkLoadDoubleTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void bulkLoadDoubleTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         double[] fieldValues = new double[ctx.getFieldCount()];
@@ -194,7 +195,7 @@
     protected CheckTuple createIntCheckTuple(int[] fieldValues, int numKeyFields) {
         RTreeCheckTuple<Integer> checkTuple = new RTreeCheckTuple<Integer>(fieldValues.length, numKeyFields);
         for (int v : fieldValues) {
-            checkTuple.add(v);
+            checkTuple.appendField(v);
         }
         return checkTuple;
     }
@@ -213,7 +214,7 @@
             fieldValues[k] = secondValue;
         }
     }
-    
+
     @Override
     protected void setIntPayloadFields(int[] fieldValues, int numKeyFields, int numFields) {
         for (int j = numKeyFields; j < numFields; j++) {
@@ -227,12 +228,12 @@
     }
 
     @Override
-    protected ArrayTupleBuilder createDeleteTupleBuilder(ITreeIndexTestContext ctx) {
+    protected ArrayTupleBuilder createDeleteTupleBuilder(IIndexTestContext ctx) {
         return new ArrayTupleBuilder(ctx.getFieldCount());
     }
 
     @Override
-    protected boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple, ITreeIndexTestContext ctx)
+    protected boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple, IIndexTestContext ctx)
             throws HyracksDataException {
         return ctx.getCheckTuples().contains(checkTuple);
     }
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexLifecycleManagerProvider.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexLifecycleManagerProvider.java
new file mode 100644
index 0000000..3a4d3e7
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexLifecycleManagerProvider.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hyracks.test.support;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+
+public class TestIndexLifecycleManagerProvider implements IIndexLifecycleManagerProvider {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIndexLifecycleManager getLifecycleManager(IHyracksTaskContext ctx) {
+        return TestStorageManagerComponentHolder.getIndexLifecycleManager(ctx);
+    }
+
+}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexRegistryProvider.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexRegistryProvider.java
deleted file mode 100644
index 27d50f5..0000000
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexRegistryProvider.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright 2009-2010 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.test.support;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-
-public class TestIndexRegistryProvider implements IIndexRegistryProvider<IIndex> {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public IndexRegistry<IIndex> getRegistry(IHyracksTaskContext ctx) {
-        return TestStorageManagerComponentHolder.getIndexRegistry(ctx);
-    }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
index ce5e989..fdb2100 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -23,8 +23,8 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+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.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.buffercache.DelayPageCleanerPolicy;
@@ -34,13 +34,20 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.storage.common.smi.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceRepositoryFactory;
 
 public class TestStorageManagerComponentHolder {
     private static IBufferCache bufferCache;
     private static IFileMapProvider fileMapProvider;
-    private static IndexRegistry<IIndex> indexRegistry;
     private static IOManager ioManager;
+    private static ILocalResourceRepository localResourceRepository;
+    private static IIndexLifecycleManager lcManager;
+    private static ResourceIdFactory resourceIdFactory;
 
     private static int pageSize;
     private static int numPages;
@@ -52,7 +59,15 @@
         TestStorageManagerComponentHolder.maxOpenFiles = maxOpenFiles;
         bufferCache = null;
         fileMapProvider = null;
-        indexRegistry = null;
+        localResourceRepository = null;
+        lcManager = null;
+    }
+
+    public synchronized static IIndexLifecycleManager getIndexLifecycleManager(IHyracksTaskContext ctx) {
+        if (lcManager == null) {
+            lcManager = new IndexLifecycleManager();
+        }
+        return lcManager;
     }
 
     public synchronized static IBufferCache getBufferCache(IHyracksTaskContext ctx) {
@@ -73,13 +88,6 @@
         return fileMapProvider;
     }
 
-    public synchronized static IndexRegistry<IIndex> getIndexRegistry(IHyracksTaskContext ctx) {
-        if (indexRegistry == null) {
-            indexRegistry = new IndexRegistry<IIndex>();
-        }
-        return indexRegistry;
-    }
-
     public synchronized static IOManager getIOManager() throws HyracksException {
         if (ioManager == null) {
             List<IODeviceHandle> devices = new ArrayList<IODeviceHandle>();
@@ -88,4 +96,31 @@
         }
         return ioManager;
     }
+
+    public synchronized static ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
+        if (localResourceRepository == null) {
+            try {
+                ILocalResourceRepositoryFactory localResourceRepositoryFactory = new TransientLocalResourceRepositoryFactory();
+                localResourceRepository = localResourceRepositoryFactory.createRepository();
+            } catch (HyracksException e) {
+                //In order not to change the IStorageManagerInterface due to the test code, throw runtime exception.
+                throw new IllegalArgumentException();
+            }
+        }
+        return localResourceRepository;
+    }
+
+    public synchronized static ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
+        if (resourceIdFactory == null) {
+            try {
+                ResourceIdFactoryProvider resourceIdFactoryFactory = new ResourceIdFactoryProvider(
+                        getLocalResourceRepository(ctx));
+                resourceIdFactory = resourceIdFactoryFactory.createResourceIdFactory();
+            } catch (HyracksException e) {
+                //In order not to change the IStorageManagerInterface due to the test code, throw runtime exception.
+                throw new IllegalArgumentException();
+            }
+        }
+        return resourceIdFactory;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
index 4059ef0..26c7861 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
@@ -15,9 +15,12 @@
 package edu.uci.ics.hyracks.test.support;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 
 public class TestStorageManagerInterface implements IStorageManagerInterface {
     private static final long serialVersionUID = 1L;
@@ -31,4 +34,14 @@
     public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
         return TestStorageManagerComponentHolder.getFileMapProvider(ctx);
     }
+
+    @Override
+    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
+        return TestStorageManagerComponentHolder.getLocalResourceRepository(ctx);
+    }
+
+	@Override
+	public ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
+		return TestStorageManagerComponentHolder.getResourceIdFactory(ctx);
+	}
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
new file mode 100644
index 0000000..bdd7433
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
@@ -0,0 +1,47 @@
+<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>
+  <artifactId>hyracks-storage-am-bloomfilter-test</artifactId>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-tests</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  	<dependency>
+  		<groupId>junit</groupId>
+  		<artifactId>junit</artifactId>
+  		<version>4.8.1</version>
+  		<type>jar</type>
+  		<scope>test</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-bloomfilter</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-test-support</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>test</scope>
+  	</dependency>
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/BloomFilterTest.java b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/BloomFilterTest.java
new file mode 100644
index 0000000..6dab32c
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/BloomFilterTest.java
@@ -0,0 +1,167 @@
+/*
+ * Copyright 2009-2010 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.bloomfilter;
+
+import java.util.ArrayList;
+import java.util.Random;
+import java.util.TreeSet;
+import java.util.logging.Level;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+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.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.util.AbstractBloomFilterTest;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+@SuppressWarnings("rawtypes")
+public class BloomFilterTest extends AbstractBloomFilterTest {
+    private final Random rnd = new Random(50);
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        super.setUp();
+    }
+
+    @Test
+    public void singleFieldTest() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TESTING BLOOM FILTER");
+        }
+
+        IBufferCache bufferCache = harness.getBufferCache();
+
+        int numElements = 100;
+        int[] keyFields = { 0 };
+
+        BloomFilter bf = new BloomFilter(bufferCache, harness.getFileMapProvider(), harness.getFileReference(),
+                keyFields);
+
+        double acceptanleFalsePositiveRate = 0.1;
+        int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
+        BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                acceptanleFalsePositiveRate);
+
+        bf.create();
+        bf.activate();
+        IIndexBulkLoader builder = bf.createBuilder(numElements, bloomFilterSpec.getNumHashes(),
+                bloomFilterSpec.getNumBucketsPerElements());
+
+        int fieldCount = 2;
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+
+        // generate keys
+        int maxKey = 1000;
+        TreeSet<Integer> uniqueKeys = new TreeSet<Integer>();
+        ArrayList<Integer> keys = new ArrayList<Integer>();
+        while (uniqueKeys.size() < numElements) {
+            int key = rnd.nextInt() % maxKey;
+            uniqueKeys.add(key);
+        }
+        for (Integer i : uniqueKeys) {
+            keys.add(i);
+        }
+
+        // Insert tuples in the bloom filter
+        for (int i = 0; i < keys.size(); ++i) {
+            TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
+            builder.add(tuple);
+        }
+        builder.end();
+
+        // Check all the inserted tuples can be found.
+
+        long[] hashes = new long[2];
+        for (int i = 0; i < keys.size(); ++i) {
+            TupleUtils.createIntegerTuple(tupleBuilder, tuple, keys.get(i), i);
+            Assert.assertTrue(bf.contains(tuple, hashes));
+        }
+
+        bf.deactivate();
+        bf.destroy();
+    }
+
+    @Test
+    public void multiFieldTest() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TESTING BLOOM FILTER");
+        }
+
+        IBufferCache bufferCache = harness.getBufferCache();
+
+        int numElements = 10000;
+        int[] keyFields = { 2, 4, 1 };
+
+        BloomFilter bf = new BloomFilter(bufferCache, harness.getFileMapProvider(), harness.getFileReference(),
+                keyFields);
+
+        double acceptanleFalsePositiveRate = 0.1;
+        int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
+        BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
+                acceptanleFalsePositiveRate);
+
+        bf.create();
+        bf.activate();
+        IIndexBulkLoader builder = bf.createBuilder(numElements, bloomFilterSpec.getNumHashes(),
+                bloomFilterSpec.getNumBucketsPerElements());
+
+        int fieldCount = 5;
+        ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+
+        int maxLength = 20;
+        ArrayList<String> s1 = new ArrayList<String>();
+        ArrayList<String> s2 = new ArrayList<String>();
+        ArrayList<String> s3 = new ArrayList<String>();
+        ArrayList<String> s4 = new ArrayList<String>();
+        for (int i = 0; i < numElements; ++i) {
+            s1.add(randomString(rnd.nextInt() % maxLength, rnd));
+            s2.add(randomString(rnd.nextInt() % maxLength, rnd));
+            s3.add(randomString(rnd.nextInt() % maxLength, rnd));
+            s4.add(randomString(rnd.nextInt() % maxLength, rnd));
+        }
+
+        for (int i = 0; i < numElements; ++i) {
+            TupleUtils.createTuple(tupleBuilder, tuple, fieldSerdes, s1.get(i), s2.get(i), i, s3.get(i), s4.get(i));
+            builder.add(tuple);
+        }
+        builder.end();
+
+        long[] hashes = new long[2];
+        for (int i = 0; i < numElements; ++i) {
+            TupleUtils.createTuple(tupleBuilder, tuple, fieldSerdes, s1.get(i), s2.get(i), i, s3.get(i), s4.get(i));
+            Assert.assertTrue(bf.contains(tuple, hashes));
+        }
+
+        bf.deactivate();
+        bf.destroy();
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java
new file mode 100644
index 0000000..284a6cb
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/MurmurHashForITupleReferenceTest.java
@@ -0,0 +1,296 @@
+/*
+ * Copyright 2009-2010 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.bloomfilter;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+import java.util.logging.Level;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.util.AbstractBloomFilterTest;
+
+@SuppressWarnings("rawtypes")
+public class MurmurHashForITupleReferenceTest extends AbstractBloomFilterTest {
+    private final static int NUM_LONG_VARS_FOR_128_BIT_HASH = 2;
+    private final static int DUMMY_FIELD = 0;
+    private final Random rnd = new Random(50);
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        super.setUp();
+    }
+
+    @Test
+    public void murmurhashONEIntegerFieldTest() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TESTING MURMUR HASH ONE INTEGER FIELD");
+        }
+
+        int fieldCount = 2;
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        TupleUtils.createIntegerTuple(tupleBuilder, tuple, rnd.nextInt());
+        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+
+        int keyFields[] = { 0 };
+        int length = getTupleSize(tuple, keyFields);
+
+        long actuals[] = new long[NUM_LONG_VARS_FOR_128_BIT_HASH];
+        MurmurHash128Bit.hash3_x64_128(tuple, keyFields, 0L, actuals);
+
+        ByteBuffer buffer;
+        byte[] array = new byte[length];
+        fillArrayWithData(array, keyFields, tuple, length);
+        buffer = ByteBuffer.wrap(array);
+
+        long[] expecteds = hash3_x64_128(buffer, 0, length, 0L);
+        Assert.assertArrayEquals(expecteds, actuals);
+    }
+
+    @Test
+    public void murmurhashTwoIntegerFieldsTest() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TESTING MURMUR HASH TWO INTEGER FIELDS");
+        }
+
+        int fieldCount = 2;
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        TupleUtils.createIntegerTuple(tupleBuilder, tuple, rnd.nextInt(), rnd.nextInt());
+        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+
+        int keyFields[] = { 0, 1 };
+        int length = getTupleSize(tuple, keyFields);
+
+        long actuals[] = new long[NUM_LONG_VARS_FOR_128_BIT_HASH];
+        MurmurHash128Bit.hash3_x64_128(tuple, keyFields, 0L, actuals);
+
+        ByteBuffer buffer;
+        byte[] array = new byte[length];
+        fillArrayWithData(array, keyFields, tuple, length);
+        buffer = ByteBuffer.wrap(array);
+
+        long[] expecteds = hash3_x64_128(buffer, 0, length, 0L);
+        Assert.assertArrayEquals(expecteds, actuals);
+    }
+
+    @Test
+    public void murmurhashOneStringFieldTest() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TESTING MURMUR HASH ONE STRING FIELD");
+        }
+
+        int fieldCount = 2;
+        ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE };
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        String s = randomString(100, rnd);
+        TupleUtils.createTuple(tupleBuilder, tuple, fieldSerdes, s);
+
+        int keyFields[] = { 0 };
+        int length = getTupleSize(tuple, keyFields);
+
+        long actuals[] = new long[NUM_LONG_VARS_FOR_128_BIT_HASH];
+        MurmurHash128Bit.hash3_x64_128(tuple, keyFields, 0L, actuals);
+
+        byte[] array = new byte[length];
+        ByteBuffer buffer;
+        fillArrayWithData(array, keyFields, tuple, length);
+        buffer = ByteBuffer.wrap(array);
+
+        long[] expecteds = hash3_x64_128(buffer, 0, length, 0L);
+        Assert.assertArrayEquals(expecteds, actuals);
+    }
+
+    @Test
+    public void murmurhashThreeStringFieldsTest() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TESTING MURMUR HASH THREE STRING FIELDS");
+        }
+
+        int fieldCount = 3;
+        ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        String s1 = randomString(40, rnd);
+        String s2 = randomString(60, rnd);
+        String s3 = randomString(20, rnd);
+        TupleUtils.createTuple(tupleBuilder, tuple, fieldSerdes, s1, s2, s3);
+
+        int keyFields[] = { 2, 0, 1 };
+        int length = getTupleSize(tuple, keyFields);
+
+        long actuals[] = new long[NUM_LONG_VARS_FOR_128_BIT_HASH];
+        MurmurHash128Bit.hash3_x64_128(tuple, keyFields, 0L, actuals);
+
+        byte[] array = new byte[length];
+        ByteBuffer buffer;
+        fillArrayWithData(array, keyFields, tuple, length);
+        buffer = ByteBuffer.wrap(array);
+
+        long[] expecteds = hash3_x64_128(buffer, 0, length, 0L);
+        Assert.assertArrayEquals(expecteds, actuals);
+    }
+
+    private void fillArrayWithData(byte[] array, int[] keyFields, ITupleReference tuple, int length) {
+        int currentFieldIndex = 0;
+        int bytePos = 0;
+        for (int i = 0; i < length; ++i) {
+            array[i] = tuple.getFieldData(DUMMY_FIELD)[tuple.getFieldStart(keyFields[currentFieldIndex]) + bytePos];
+            ++bytePos;
+            if (tuple.getFieldLength(keyFields[currentFieldIndex]) == bytePos) {
+                ++currentFieldIndex;
+                bytePos = 0;
+            }
+        }
+    }
+
+    private int getTupleSize(ITupleReference tuple, int[] keyFields) {
+        int length = 0;
+        for (int i = 0; i < keyFields.length; ++i) {
+            length += tuple.getFieldLength(keyFields[i]);
+        }
+        return length;
+    }
+
+    /**
+     * The hash3_x64_128 and getblock functions are borrowed from cassandra source code for testing purpose
+     **/
+    protected static long getblock(ByteBuffer key, int offset, int index) {
+        int i_8 = index << 3;
+        int blockOffset = offset + i_8;
+        return ((long) key.get(blockOffset + 0) & 0xff) + (((long) key.get(blockOffset + 1) & 0xff) << 8)
+                + (((long) key.get(blockOffset + 2) & 0xff) << 16) + (((long) key.get(blockOffset + 3) & 0xff) << 24)
+                + (((long) key.get(blockOffset + 4) & 0xff) << 32) + (((long) key.get(blockOffset + 5) & 0xff) << 40)
+                + (((long) key.get(blockOffset + 6) & 0xff) << 48) + (((long) key.get(blockOffset + 7) & 0xff) << 56);
+    }
+
+    public static long[] hash3_x64_128(ByteBuffer key, int offset, int length, long seed) {
+        final int nblocks = length >> 4; // Process as 128-bit blocks.
+
+        long h1 = seed;
+        long h2 = seed;
+
+        long c1 = 0x87c37b91114253d5L;
+        long c2 = 0x4cf5ad432745937fL;
+
+        //----------
+        // body
+
+        for (int i = 0; i < nblocks; i++) {
+            long k1 = getblock(key, offset, i * 2 + 0);
+            long k2 = getblock(key, offset, i * 2 + 1);
+
+            k1 *= c1;
+            k1 = MurmurHash128Bit.rotl64(k1, 31);
+            k1 *= c2;
+            h1 ^= k1;
+
+            h1 = MurmurHash128Bit.rotl64(h1, 27);
+            h1 += h2;
+            h1 = h1 * 5 + 0x52dce729;
+
+            k2 *= c2;
+            k2 = MurmurHash128Bit.rotl64(k2, 33);
+            k2 *= c1;
+            h2 ^= k2;
+
+            h2 = MurmurHash128Bit.rotl64(h2, 31);
+            h2 += h1;
+            h2 = h2 * 5 + 0x38495ab5;
+        }
+
+        //----------
+        // tail
+
+        // Advance offset to the unprocessed tail of the data.
+        offset += nblocks * 16;
+
+        long k1 = 0;
+        long k2 = 0;
+
+        switch (length & 15) {
+            case 15:
+                k2 ^= ((long) key.get(offset + 14)) << 48;
+            case 14:
+                k2 ^= ((long) key.get(offset + 13)) << 40;
+            case 13:
+                k2 ^= ((long) key.get(offset + 12)) << 32;
+            case 12:
+                k2 ^= ((long) key.get(offset + 11)) << 24;
+            case 11:
+                k2 ^= ((long) key.get(offset + 10)) << 16;
+            case 10:
+                k2 ^= ((long) key.get(offset + 9)) << 8;
+            case 9:
+                k2 ^= ((long) key.get(offset + 8)) << 0;
+                k2 *= c2;
+                k2 = MurmurHash128Bit.rotl64(k2, 33);
+                k2 *= c1;
+                h2 ^= k2;
+
+            case 8:
+                k1 ^= ((long) key.get(offset + 7)) << 56;
+            case 7:
+                k1 ^= ((long) key.get(offset + 6)) << 48;
+            case 6:
+                k1 ^= ((long) key.get(offset + 5)) << 40;
+            case 5:
+                k1 ^= ((long) key.get(offset + 4)) << 32;
+            case 4:
+                k1 ^= ((long) key.get(offset + 3)) << 24;
+            case 3:
+                k1 ^= ((long) key.get(offset + 2)) << 16;
+            case 2:
+                k1 ^= ((long) key.get(offset + 1)) << 8;
+            case 1:
+                k1 ^= ((long) key.get(offset));
+                k1 *= c1;
+                k1 = MurmurHash128Bit.rotl64(k1, 31);
+                k1 *= c2;
+                h1 ^= k1;
+        };
+
+        //----------
+        // finalization
+
+        h1 ^= length;
+        h2 ^= length;
+
+        h1 += h2;
+        h2 += h1;
+
+        h1 = MurmurHash128Bit.fmix(h1);
+        h2 = MurmurHash128Bit.fmix(h2);
+
+        h1 += h2;
+        h2 += h1;
+
+        return (new long[] { h1, h2 });
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/util/AbstractBloomFilterTest.java b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/util/AbstractBloomFilterTest.java
new file mode 100644
index 0000000..9b857a6
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/util/AbstractBloomFilterTest.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2010 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.bloomfilter.util;
+
+import java.util.Random;
+import java.util.logging.Logger;
+
+import org.junit.After;
+import org.junit.Before;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractBloomFilterTest {
+    protected final Logger LOGGER = Logger.getLogger(BloomFilterTestHarness.class.getName());
+
+    protected final BloomFilterTestHarness harness;
+
+    public AbstractBloomFilterTest() {
+        harness = new BloomFilterTestHarness();
+    }
+
+    public AbstractBloomFilterTest(int pageSize, int numPages, int maxOpenFiles, int hyracksFrameSize) {
+        harness = new BloomFilterTestHarness(pageSize, numPages, maxOpenFiles, hyracksFrameSize);
+    }
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    public static String randomString(int length, Random random) {
+        char[] chars = "abcdefghijklmnopqrstuvwxyz".toCharArray();
+        StringBuilder strBuilder = new StringBuilder();
+        for (int i = 0; i < length; ++i) {
+            char c = chars[random.nextInt(chars.length)];
+            strBuilder.append(c);
+        }
+        return strBuilder.toString();
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/util/BloomFilterTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/util/BloomFilterTestHarness.java
new file mode 100644
index 0000000..8fac122
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/src/test/java/edu/uci/ics/hyracks/storage/am/bloomfilter/util/BloomFilterTestHarness.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2009-2010 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.bloomfilter.util;
+
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Random;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public class BloomFilterTestHarness {
+
+    private static final long RANDOM_SEED = 50;
+
+    protected final int pageSize;
+    protected final int numPages;
+    protected final int maxOpenFiles;
+    protected final int hyracksFrameSize;
+
+    protected IHyracksTaskContext ctx;
+    protected IBufferCache bufferCache;
+    protected IFileMapProvider fileMapProvider;
+    protected FileReference file;
+
+    protected final Random rnd = new Random();
+    protected final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    protected final String tmpDir = System.getProperty("java.io.tmpdir");
+    protected final String sep = System.getProperty("file.separator");
+    protected String fileName;
+
+    public BloomFilterTestHarness() {
+        this.pageSize = AccessMethodTestsConfig.BLOOM_FILTER_PAGE_SIZE;
+        this.numPages = AccessMethodTestsConfig.BLOOM_FILTER_NUM_PAGES;
+        this.maxOpenFiles = AccessMethodTestsConfig.BLOOM_FILTER_MAX_OPEN_FILES;
+        this.hyracksFrameSize = AccessMethodTestsConfig.BLOOM_FILTER_HYRACKS_FRAME_SIZE;
+    }
+
+    public BloomFilterTestHarness(int pageSize, int numPages, int maxOpenFiles, int hyracksFrameSize) {
+        this.pageSize = pageSize;
+        this.numPages = numPages;
+        this.maxOpenFiles = maxOpenFiles;
+        this.hyracksFrameSize = hyracksFrameSize;
+    }
+
+    public void setUp() throws HyracksDataException {
+        fileName = tmpDir + sep + simpleDateFormat.format(new Date());
+        ctx = TestUtils.create(getHyracksFrameSize());
+        TestStorageManagerComponentHolder.init(pageSize, numPages, maxOpenFiles);
+        bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        fileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        file = new FileReference(new File(fileName));
+        rnd.setSeed(RANDOM_SEED);
+    }
+
+    public void tearDown() throws HyracksDataException {
+        bufferCache.close();
+        file.delete();
+    }
+
+    public IHyracksTaskContext getHyracksTaskContext() {
+        return ctx;
+    }
+
+    public IBufferCache getBufferCache() {
+        return bufferCache;
+    }
+
+    public IFileMapProvider getFileMapProvider() {
+        return fileMapProvider;
+    }
+
+    public FileReference getFileReference() {
+        return file;
+    }
+
+    public String getFileName() {
+        return fileName;
+    }
+
+    public Random getRandom() {
+        return rnd;
+    }
+
+    public int getPageSize() {
+        return pageSize;
+    }
+
+    public int getNumPages() {
+        return numPages;
+    }
+
+    public int getHyracksFrameSize() {
+        return hyracksFrameSize;
+    }
+
+    public int getMaxOpenFiles() {
+        return maxOpenFiles;
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
index d0bb883..cf2f9e6 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
@@ -1,8 +1,6 @@
 <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>hyracks-storage-am-btree-test</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-storage-am-btree-test</name>
 
   <parent>
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeBulkLoadTest.java
index 11c47c7..5417ca1 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeBulkLoadTest.java
@@ -22,21 +22,18 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexBulkLoadTest;
-import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestContext;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
 
-@SuppressWarnings("rawtypes")
 public class BTreeBulkLoadTest extends OrderedIndexBulkLoadTest {
 
+    private final BTreeTestHarness harness = new BTreeTestHarness();
+
     public BTreeBulkLoadTest() {
         super(BTreeTestHarness.LEAF_FRAMES_TO_TEST, 1);
     }
 
-    private final BTreeTestHarness harness = new BTreeTestHarness();
-
     @Before
     public void setUp() throws HyracksDataException {
         harness.setUp();
@@ -47,11 +44,12 @@
         harness.tearDown();
     }
 
+    @SuppressWarnings("rawtypes")
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return BTreeTestContext.create(harness.getBufferCache(), harness.getBTreeFileId(), fieldSerdes, numKeys,
-                leafType);
+        return BTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, numKeys, leafType);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeDeleteTest.java
index 0205540..10cd59f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeDeleteTest.java
@@ -22,21 +22,18 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexDeleteTest;
-import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestContext;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
 
-@SuppressWarnings("rawtypes")
 public class BTreeDeleteTest extends OrderedIndexDeleteTest {
 
+    private final BTreeTestHarness harness = new BTreeTestHarness();
+
     public BTreeDeleteTest() {
         super(BTreeTestHarness.LEAF_FRAMES_TO_TEST);
     }
 
-    private final BTreeTestHarness harness = new BTreeTestHarness();
-
     @Before
     public void setUp() throws HyracksDataException {
         harness.setUp();
@@ -47,11 +44,12 @@
         harness.tearDown();
     }
 
+    @SuppressWarnings("rawtypes")
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return BTreeTestContext.create(harness.getBufferCache(), harness.getBTreeFileId(), fieldSerdes, numKeys,
-                leafType);
+        return BTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, numKeys, leafType);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
index f4f8b12..c02d53d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
@@ -21,7 +21,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.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexExamplesTest;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
@@ -40,13 +39,11 @@
     public void tearDown() throws HyracksDataException {
         harness.tearDown();
     }
-    
-    protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories) throws TreeIndexException {
-        return BTreeUtils.createBTree(harness.getBufferCache(), harness.getOpCallback(), typeTraits, cmpFactories,
-                BTreeLeafFrameType.REGULAR_NSM);
+
+    protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields) throws TreeIndexException {
+        return BTreeUtils.createBTree(harness.getBufferCache(), harness.getFileMapProvider(), typeTraits, cmpFactories,
+                BTreeLeafFrameType.REGULAR_NSM, harness.getFileReference());
     }
-    
-    protected int getIndexFileId() {
-        return harness.getBTreeFileId();
-    }
+
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeInsertTest.java
index 0b6cf4d..efe7579 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeInsertTest.java
@@ -22,31 +22,27 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexInsertTest;
-import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestContext;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
 
 /**
- * Tests the BTree insert operation with strings and integer fields using
- * various numbers of key and payload fields.
- * 
- * Each tests first fills a BTree with randomly generated tuples. We compare the
- * following operations against expected results: 1. Point searches for all
- * tuples. 2. Ordered scan. 3. Disk-order scan. 4. Range search (and prefix
- * search for composite keys).
- * 
+ * Tests the BTree insert operation with strings and integer fields using 
+ * various numbers of key and payload fields. Each tests first fills a BTree with 
+ * randomly generated tuples. We compare the following operations against expected results: 
+ *      1) Point searches for all tuples 
+ *      2) Ordered scan
+ *      3) Disk-order scan
+ *      4) Range search (and prefix search for composite keys)
  */
-@SuppressWarnings("rawtypes")
 public class BTreeInsertTest extends OrderedIndexInsertTest {
 
+    private final BTreeTestHarness harness = new BTreeTestHarness();
+
     public BTreeInsertTest() {
         super(BTreeTestHarness.LEAF_FRAMES_TO_TEST);
     }
 
-    private final BTreeTestHarness harness = new BTreeTestHarness();
-
     @Before
     public void setUp() throws HyracksDataException {
         harness.setUp();
@@ -57,11 +53,12 @@
         harness.tearDown();
     }
 
+    @SuppressWarnings("rawtypes")
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return BTreeTestContext.create(harness.getBufferCache(), harness.getBTreeFileId(), fieldSerdes, numKeys,
-                leafType);
+        return BTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, numKeys, leafType);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeLifecycleTest.java
new file mode 100644
index 0000000..1445d2c
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeLifecycleTest.java
@@ -0,0 +1,71 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.common.AbstractIndexLifecycleTest;
+import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+
+public class BTreeLifecycleTest extends AbstractIndexLifecycleTest {
+    private final BTreeTestHarness harness = new BTreeTestHarness();
+    private final TreeIndexTestUtils titu = new OrderedIndexTestUtils();
+
+    @SuppressWarnings("rawtypes")
+    private final ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE };
+
+    private ITreeIndexFrame frame = null;
+
+    @SuppressWarnings("rawtypes")
+    private IIndexTestContext<? extends CheckTuple> testCtx;
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        testCtx = BTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, fieldSerdes.length, BTreeLeafFrameType.REGULAR_NSM);
+        index = testCtx.getIndex();
+    }
+
+    @Override
+    public void tearDown() throws HyracksDataException {
+        testCtx.getIndex().deactivate();
+        testCtx.getIndex().destroy();
+        harness.tearDown();
+    }
+
+    @Override
+    protected boolean persistentStateExists() {
+        return harness.getFileReference().getFile().exists();
+    }
+
+    @Override
+    protected boolean isEmptyIndex() throws HyracksDataException {
+        BTree btree = (BTree) testCtx.getIndex();
+        if (frame == null) {
+            frame = btree.getInteriorFrameFactory().createFrame();
+        }
+        return btree.isEmptyTree(frame);
+    }
+
+    @Override
+    protected void performInsertions() throws Exception {
+        titu.insertIntTuples(testCtx, 10, harness.getRandom());
+    }
+
+    @Override
+    protected void checkInsertions() throws Exception {
+        titu.checkScan(testCtx);
+    }
+
+    @Override
+    protected void clearCheckableInsertions() throws Exception {
+        testCtx.getCheckTuples().clear();
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java
new file mode 100644
index 0000000..b5cbca3
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeModificationOperationCallbackTest.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+
+public class BTreeModificationOperationCallbackTest extends AbstractModificationOperationCallbackTest {
+    private final BTreeTestHarness harness;
+
+    public BTreeModificationOperationCallbackTest() {
+        harness = new BTreeTestHarness();
+    }
+
+    @Override
+    protected void createIndexInstance() throws Exception {
+        index = BTreeUtils.createBTree(harness.getBufferCache(), harness.getFileMapProvider(),
+                SerdeUtils.serdesToTypeTraits(keySerdes),
+                SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), BTreeLeafFrameType.REGULAR_NSM,
+                harness.getFileReference());
+    }
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        super.setup();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        harness.tearDown();
+    }
+
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSearchCursorTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSearchCursorTest.java
index 4003cf1..ad0b21e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSearchCursorTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSearchCursorTest.java
@@ -24,6 +24,7 @@
 import java.util.TreeSet;
 import java.util.logging.Level;
 
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -47,29 +48,24 @@
 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.AbstractBTreeTest;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
 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.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-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.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 
 public class BTreeSearchCursorTest extends AbstractBTreeTest {
-    // Declare fields
-    int fieldCount = 2;
-    ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
-
-    TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
-    ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-    ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
-
-    Random rnd = new Random(50);
+    private final int fieldCount = 2;
+    private final ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+    private final TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+    private final ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+    private final Random rnd = new Random(50);
 
     @Before
     public void setUp() throws HyracksDataException {
@@ -85,8 +81,7 @@
         }
 
         IBufferCache bufferCache = harness.getBufferCache();
-        int btreeFileId = harness.getBTreeFileId();
-        
+
         // declare keys
         int keyFieldCount = 1;
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
@@ -100,14 +95,16 @@
 
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
 
-        BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, fieldCount, cmpFactories, freePageManager, interiorFrameFactory, leafFrameFactory);
-        btree.create(btreeFileId);
-        btree.open(btreeFileId);
+        BTree btree = new BTree(bufferCache, harness.getFileMapProvider(), freePageManager, interiorFrameFactory,
+                leafFrameFactory, cmpFactories, fieldCount, harness.getFileReference());
+        btree.create();
+        btree.activate();
 
         ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
-        ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        ITreeIndexAccessor indexAccessor = btree.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
 
         // generate keys
         int numKeys = 50;
@@ -136,18 +133,21 @@
             }
         }
 
-        // btree.printTree(leafFrame, interiorFrame, recDescSers);
-
         int minSearchKey = -100;
         int maxSearchKey = 100;
 
         // forward searches
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false);
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, false);
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, false, false);
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false);
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true,
+                true, false));
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false,
+                true, false));
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true,
+                false, false));
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true,
+                true, false));
 
-        btree.close();
+        btree.deactivate();
+        btree.destroy();
     }
 
     @Test
@@ -157,8 +157,7 @@
         }
 
         IBufferCache bufferCache = harness.getBufferCache();
-        int btreeFileId = harness.getBTreeFileId();
-        
+
         // declare keys
         int keyFieldCount = 2;
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
@@ -173,14 +172,16 @@
 
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
 
-        BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, fieldCount, cmpFactories, freePageManager, interiorFrameFactory, leafFrameFactory);
-        btree.create(btreeFileId);
-        btree.open(btreeFileId);
+        BTree btree = new BTree(bufferCache, harness.getFileMapProvider(), freePageManager, interiorFrameFactory,
+                leafFrameFactory, cmpFactories, fieldCount, harness.getFileReference());
+        btree.create();
+        btree.activate();
 
         ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
-        ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        ITreeIndexAccessor indexAccessor = btree.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
 
         // generate keys
         int numKeys = 50;
@@ -206,18 +207,21 @@
             }
         }
 
-        // btree.printTree(leafFrame, interiorFrame, recDescSers);
-
         int minSearchKey = -100;
         int maxSearchKey = 100;
 
         // forward searches
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false);
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, false);
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, false, false);
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false);
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true,
+                true, false));
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false,
+                true, false));
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true,
+                false, false));
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true,
+                true, false));
 
-        btree.close();
+        btree.deactivate();
+        btree.destroy();
     }
 
     @Test
@@ -227,8 +231,7 @@
         }
 
         IBufferCache bufferCache = harness.getBufferCache();
-        int btreeFileId = harness.getBTreeFileId();
-        
+
         // declare keys
         int keyFieldCount = 2;
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
@@ -243,14 +246,16 @@
 
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
 
-        BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, fieldCount, cmpFactories, freePageManager, interiorFrameFactory, leafFrameFactory);
-        btree.create(btreeFileId);
-        btree.open(btreeFileId);
+        BTree btree = new BTree(bufferCache, harness.getFileMapProvider(), freePageManager, interiorFrameFactory,
+                leafFrameFactory, cmpFactories, fieldCount, harness.getFileReference());
+        btree.create();
+        btree.activate();
 
         ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
-        ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        ITreeIndexAccessor indexAccessor = btree.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
 
         // generate keys
         int numKeys = 50;
@@ -276,22 +281,25 @@
             }
         }
 
-        // btree.printTree(leafFrame, interiorFrame, recDescSers);
-
         int minSearchKey = -100;
         int maxSearchKey = 100;
 
         // forward searches
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false);
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false, true, false);
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, false, false);
-        performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true, true, false);
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true,
+                true, false));
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, false,
+                true, false));
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true,
+                false, false));
+        Assert.assertTrue(performSearches(keys, btree, leafFrame, interiorFrame, minSearchKey, maxSearchKey, true,
+                true, false));
 
-        btree.close();
+        btree.deactivate();
+        btree.destroy();
     }
 
-    public RangePredicate createRangePredicate(int lk, int hk, boolean lowKeyInclusive,
-            boolean highKeyInclusive) throws HyracksDataException {
+    public RangePredicate createRangePredicate(int lk, int hk, boolean lowKeyInclusive, boolean highKeyInclusive)
+            throws HyracksDataException {
 
         // create tuplereferences for search keys
         ITupleReference lowKey = TupleUtils.createIntegerTuple(lk);
@@ -301,8 +309,8 @@
         searchCmps[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator();
         MultiComparator searchCmp = new MultiComparator(searchCmps);
 
-        RangePredicate rangePred = new RangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive,
-                searchCmp, searchCmp);
+        RangePredicate rangePred = new RangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive, searchCmp,
+                searchCmp);
         return rangePred;
     }
 
@@ -345,9 +353,9 @@
                 int highKey = j;
 
                 ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
-                RangePredicate rangePred = createRangePredicate(lowKey, highKey, lowKeyInclusive,
-                        highKeyInclusive);
-                ITreeIndexAccessor indexAccessor = btree.createAccessor();
+                RangePredicate rangePred = createRangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive);
+                ITreeIndexAccessor indexAccessor = btree.createAccessor(TestOperationCallback.INSTANCE,
+                        TestOperationCallback.INSTANCE);
                 indexAccessor.search(rangeCursor, rangePred);
 
                 try {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java
new file mode 100644
index 0000000..037d992
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSearchOperationCallbackTest.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+
+public class BTreeSearchOperationCallbackTest extends AbstractSearchOperationCallbackTest {
+    private final BTreeTestHarness harness;
+
+    public BTreeSearchOperationCallbackTest() {
+        harness = new BTreeTestHarness();
+    }
+
+    @Override
+    protected void createIndexInstance() throws Exception {
+        index = BTreeUtils.createBTree(harness.getBufferCache(), harness.getFileMapProvider(),
+                SerdeUtils.serdesToTypeTraits(keySerdes),
+                SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), BTreeLeafFrameType.REGULAR_NSM,
+                harness.getFileReference());
+    }
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        super.setup();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        harness.tearDown();
+    }
+
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
index c33b4e9..57535b2 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.hyracks.storage.am.btree;
 
 import java.io.DataOutput;
-import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.logging.Level;
@@ -14,7 +13,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -28,6 +26,7 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.btree.util.AbstractBTreeTest;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -36,7 +35,6 @@
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexBufferCacheWarmup;
 import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStats;
@@ -52,18 +50,15 @@
     private static final int NUM_PAGES = 1000;
     private static final int MAX_OPEN_FILES = 10;
     private static final int HYRACKS_FRAME_SIZE = 128;
-    private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+
+    private final IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 
     @Test
     public void test01() throws Exception {
 
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
-        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
-        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
-        FileReference file = new FileReference(new File(harness.getFileName()));
-        bufferCache.createFile(file);
-        int fileId = fmp.lookupFileId(file);
-        bufferCache.openFile(fileId);
+        IBufferCache bufferCache = harness.getBufferCache();
+        IFileMapProvider fmp = harness.getFileMapProvider();
 
         // declare fields
         int fieldCount = 2;
@@ -87,9 +82,10 @@
 
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
 
-        BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, fieldCount, cmpFactories, freePageManager, interiorFrameFactory, leafFrameFactory);
-        btree.create(fileId);
-        btree.open(fileId);
+        BTree btree = new BTree(bufferCache, fmp, freePageManager, interiorFrameFactory, leafFrameFactory,
+                cmpFactories, fieldCount, harness.getFileReference());
+        btree.create();
+        btree.activate();
 
         Random rnd = new Random();
         rnd.setSeed(50);
@@ -112,7 +108,8 @@
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        ITreeIndexAccessor indexAccessor = btree.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         // 10000
         for (int i = 0; i < 100000; i++) {
 
@@ -145,6 +142,7 @@
             }
         }
 
+        int fileId = fmp.lookupFileId(harness.getFileReference());
         TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
                 btree.getRootPageId());
         TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
@@ -156,8 +154,8 @@
                 fileId);
         bufferCacheWarmup.warmup(leafFrame, metaFrame, new int[] { 1, 2 }, new int[] { 2, 5 });
 
-        btree.close();
-        bufferCache.closeFile(fileId);
+        btree.deactivate();
+        btree.destroy();
         bufferCache.close();
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpdateSearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
index 2b03a6a..f01799d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
@@ -22,6 +22,7 @@
 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.AbstractBTreeTest;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
@@ -30,19 +31,16 @@
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 
-@SuppressWarnings("rawtypes")
 public class BTreeUpdateSearchTest extends AbstractBTreeTest {
 
     // Update scan test on fixed-length tuples.
     @Test
     public void test01() throws Exception {
         IBufferCache bufferCache = harness.getBufferCache();
-        int btreeFileId = harness.getBTreeFileId();
-        
+
         // declare fields
         int fieldCount = 2;
         ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
@@ -54,6 +52,7 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
+        @SuppressWarnings("rawtypes")
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE };
 
@@ -65,9 +64,10 @@
         IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
 
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
-        BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, fieldCount, cmpFactories, freePageManager, interiorFrameFactory, leafFrameFactory);
-        btree.create(btreeFileId);
-        btree.open(btreeFileId);
+        BTree btree = new BTree(bufferCache, harness.getFileMapProvider(), freePageManager, interiorFrameFactory,
+                leafFrameFactory, cmpFactories, fieldCount, harness.getFileReference());
+        btree.create();
+        btree.activate();
 
         Random rnd = new Random();
         rnd.setSeed(50);
@@ -80,7 +80,8 @@
 
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference insertTuple = new ArrayTupleReference();
-        ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        ITreeIndexAccessor indexAccessor = btree.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
 
         int numInserts = 10000;
         for (int i = 0; i < numInserts; i++) {
@@ -149,6 +150,7 @@
         } finally {
             scanCursor.close();
         }
-        btree.close();
+        btree.deactivate();
+        btree.destroy();
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpdateTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpdateTest.java
index c3b56d5..89bb50e5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpdateTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpdateTest.java
@@ -22,21 +22,18 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
-import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexUpdateTest;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestContext;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
 
-@SuppressWarnings("rawtypes")
 public class BTreeUpdateTest extends OrderedIndexUpdateTest {
 
+    private final BTreeTestHarness harness = new BTreeTestHarness();
+
     public BTreeUpdateTest() {
         super(BTreeTestHarness.LEAF_FRAMES_TO_TEST);
     }
 
-    private final BTreeTestHarness harness = new BTreeTestHarness();
-
     @Before
     public void setUp() throws HyracksDataException {
         harness.setUp();
@@ -47,11 +44,12 @@
         harness.tearDown();
     }
 
+    @SuppressWarnings("rawtypes")
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return BTreeTestContext.create(harness.getBufferCache(), harness.getBTreeFileId(), fieldSerdes, numKeys,
-                leafType);
+        return BTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, numKeys, leafType);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpsertTest.java
index 6e14607..ab32156 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeUpsertTest.java
@@ -27,16 +27,14 @@
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
 
 /**
- * Tests the BTree insert operation with strings and integer fields using
- * various numbers of key and payload fields.
- * 
- * Each tests first fills a BTree with randomly generated tuples. We compare the
- * following operations against expected results: 1. Point searches for all
- * tuples. 2. Ordered scan. 3. Disk-order scan. 4. Range search (and prefix
- * search for composite keys).
- * 
+ * Tests the BTree insert operation with strings and integer fields using 
+ * various numbers of key and payload fields. Each tests first fills a BTree with 
+ * randomly generated tuples. We compare the following operations against expected results: 
+ *      1) Point searches for all tuples 
+ *      2) Ordered scan
+ *      3) Disk-order scan
+ *      4) Range search (and prefix search for composite keys)
  */
-@SuppressWarnings("rawtypes")
 public class BTreeUpsertTest extends OrderedIndexUpsertTest {
 
     public BTreeUpsertTest() {
@@ -55,11 +53,12 @@
         harness.tearDown();
     }
 
+    @SuppressWarnings("rawtypes")
     @Override
     protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception {
-        return BTreeTestContext.create(harness.getBufferCache(), harness.getBTreeFileId(), fieldSerdes, numKeys,
-                leafType);
+        return BTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, numKeys, leafType);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
index d61d16a..1bf511e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
@@ -48,19 +48,19 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
-@SuppressWarnings("rawtypes")
 public class FieldPrefixNSMTest extends AbstractBTreeTest {
 
-    private static final int PAGE_SIZE = 32768; // 32K
+    private static final int PAGE_SIZE = 32768;
     private static final int NUM_PAGES = 40;
     private static final int MAX_OPEN_FILES = 10;
     private static final int HYRACKS_FRAME_SIZE = 128;
 
-    public FieldPrefixNSMTest() {        
+    public FieldPrefixNSMTest() {
         super(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES, HYRACKS_FRAME_SIZE);
     }
-    
+
     private ITupleReference createTuple(IHyracksTaskContext ctx, int f0, int f1, int f2, boolean print)
             throws HyracksDataException {
         if (print) {
@@ -73,7 +73,8 @@
         FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
         ArrayTupleBuilder tb = new ArrayTupleBuilder(3);
         DataOutput dos = tb.getDataOutput();
-        
+
+        @SuppressWarnings("rawtypes")
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
@@ -98,8 +99,8 @@
     }
 
     @Test
-    public void test01() throws Exception {        
-        
+    public void test01() throws Exception {
+
         // declare fields
         int fieldCount = 3;
         ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
@@ -116,6 +117,7 @@
         MultiComparator cmp = new MultiComparator(cmps);
 
         // just for printing
+        @SuppressWarnings("rawtypes")
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
 
@@ -123,7 +125,10 @@
         rnd.setSeed(50);
 
         IBufferCache bufferCache = harness.getBufferCache();
-        int btreeFileId = harness.getBTreeFileId();
+        IFileMapProvider fileMapProvider = harness.getFileMapProvider();
+        bufferCache.createFile(harness.getFileReference());
+        int btreeFileId = fileMapProvider.lookupFileId(harness.getFileReference());
+        bufferCache.openFile(btreeFileId);
         IHyracksTaskContext ctx = harness.getHyracksTaskContext();
         ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(btreeFileId, 0), false);
         try {
@@ -220,6 +225,8 @@
 
         } finally {
             bufferCache.unpin(page);
+            bufferCache.closeFile(btreeFileId);
+            bufferCache.close();
         }
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
index 9ec64b5..d273a12 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
@@ -29,7 +29,7 @@
 import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 import edu.uci.ics.hyracks.storage.common.sync.LatchType;
 
-public class StorageManagerTest extends AbstractBTreeTest {	
+public class StorageManagerTest extends AbstractBTreeTest {
     public class PinnedLatchedPage {
         public final ICachedPage page;
         public final LatchType latch;
@@ -43,7 +43,10 @@
     }
 
     public enum FileAccessType {
-        FTA_READONLY, FTA_WRITEONLY, FTA_MIXED, FTA_UNLATCHED
+        FTA_READONLY,
+        FTA_WRITEONLY,
+        FTA_MIXED,
+        FTA_UNLATCHED
     }
 
     public class FileAccessWorker implements Runnable {
@@ -249,11 +252,16 @@
     }
 
     @Test
-    public void oneThreadOneFileTest() throws Exception { 
-		Thread worker = new Thread(new FileAccessWorker(0,
-				harness.getBufferCache(), FileAccessType.FTA_UNLATCHED,
-				harness.getBTreeFileId(), 10, 10, 100, 10, 0));
+    public void oneThreadOneFileTest() throws Exception {
+        IBufferCache bufferCache = harness.getBufferCache();
+        bufferCache.createFile(harness.getFileReference());
+        int btreeFileId = harness.getFileMapProvider().lookupFileId(harness.getFileReference());
+        bufferCache.openFile(btreeFileId);
+        Thread worker = new Thread(new FileAccessWorker(0, harness.getBufferCache(), FileAccessType.FTA_UNLATCHED,
+                btreeFileId, 10, 10, 100, 10, 0));
         worker.start();
         worker.join();
+        bufferCache.closeFile(btreeFileId);
+        bufferCache.close();
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java
index 596fa31..3f38c05 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java
@@ -24,18 +24,18 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestWorkerFactory;
-import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.ProbabilityHelper;
 
 public class BTreeMultiThreadTest extends OrderedIndexMultiThreadTest {
 
-    private BTreeTestHarness harness = new BTreeTestHarness();
-    
-    private BTreeTestWorkerFactory workerFactory = new BTreeTestWorkerFactory();
-    
+    private final BTreeTestHarness harness = new BTreeTestHarness();
+    private final BTreeTestWorkerFactory workerFactory = new BTreeTestWorkerFactory();
+
     @Override
     protected void setUp() throws HyracksDataException {
         harness.setUp();
@@ -47,41 +47,45 @@
     }
 
     @Override
-    protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories) throws TreeIndexException {
-        return BTreeUtils.createBTree(harness.getBufferCache(), harness.getOpCallback(), typeTraits, cmpFactories, BTreeLeafFrameType.REGULAR_NSM);
+    protected ITreeIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields) throws TreeIndexException {
+        return BTreeUtils.createBTree(harness.getBufferCache(), harness.getFileMapProvider(), typeTraits, cmpFactories,
+                BTreeLeafFrameType.REGULAR_NSM, harness.getFileReference());
     }
 
     @Override
-    protected ITreeIndexTestWorkerFactory getWorkerFactory() {
+    protected IIndexTestWorkerFactory getWorkerFactory() {
         return workerFactory;
     }
 
     @Override
     protected ArrayList<TestWorkloadConf> getTestWorkloadConf() {
         ArrayList<TestWorkloadConf> workloadConfs = new ArrayList<TestWorkloadConf>();
-        
+
         // Insert only workload.
         TestOperation[] insertOnlyOps = new TestOperation[] { TestOperation.INSERT };
-        workloadConfs.add(new TestWorkloadConf(insertOnlyOps, getUniformOpProbs(insertOnlyOps)));
-        
-        // Inserts mixed with point searches and scans.
-        TestOperation[] insertSearchOnlyOps = new TestOperation[] { TestOperation.INSERT, TestOperation.POINT_SEARCH, TestOperation.SCAN, TestOperation.DISKORDER_SCAN };
-        workloadConfs.add(new TestWorkloadConf(insertSearchOnlyOps, getUniformOpProbs(insertSearchOnlyOps)));
-        
-        // Inserts, updates, deletes, and upserts.        
-        TestOperation[] insertDeleteUpdateUpsertOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE, TestOperation.UPDATE, TestOperation.UPSERT };
-        workloadConfs.add(new TestWorkloadConf(insertDeleteUpdateUpsertOps, getUniformOpProbs(insertDeleteUpdateUpsertOps)));
-        
-        // All operations mixed.
-        TestOperation[] allOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE, TestOperation.UPDATE, TestOperation.UPSERT, TestOperation.POINT_SEARCH, TestOperation.SCAN, TestOperation.DISKORDER_SCAN };
-        workloadConfs.add(new TestWorkloadConf(allOps, getUniformOpProbs(allOps)));
-        
-        return workloadConfs;
-    }
+        workloadConfs.add(new TestWorkloadConf(insertOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertOnlyOps.length)));
 
-    @Override
-    protected int getFileId() {
-        return harness.getBTreeFileId();
+        // Inserts mixed with point searches and scans.
+        TestOperation[] insertSearchOnlyOps = new TestOperation[] { TestOperation.INSERT, TestOperation.POINT_SEARCH,
+                TestOperation.SCAN, TestOperation.DISKORDER_SCAN };
+        workloadConfs.add(new TestWorkloadConf(insertSearchOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertSearchOnlyOps.length)));
+
+        // Inserts, updates, deletes, and upserts.        
+        TestOperation[] insertDeleteUpdateUpsertOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.UPDATE, TestOperation.UPSERT };
+        workloadConfs.add(new TestWorkloadConf(insertDeleteUpdateUpsertOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteUpdateUpsertOps.length)));
+
+        // All operations mixed.
+        TestOperation[] allOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.UPDATE, TestOperation.UPSERT, TestOperation.POINT_SEARCH, TestOperation.SCAN,
+                TestOperation.DISKORDER_SCAN };
+        workloadConfs.add(new TestWorkloadConf(allOps, ProbabilityHelper.getUniformProbDist(allOps.length)));
+
+        return workloadConfs;
     }
 
     @Override
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 7d8de7d..be8dc5d 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
@@ -24,37 +24,37 @@
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeNotUpdateableException;
 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.AbstractTreeIndexTestWorker;
+import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
-public class BTreeTestWorker extends AbstractTreeIndexTestWorker {
-    
+public class BTreeTestWorker extends AbstractIndexTestWorker {
+
     private final BTree btree;
     private final int numKeyFields;
     private final ArrayTupleBuilder deleteTb;
     private final ArrayTupleReference deleteTuple = new ArrayTupleReference();
-    
-    public BTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, ITreeIndex index, int numBatches) {
+
+    public BTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
         super(dataGen, opSelector, index, numBatches);
         btree = (BTree) index;
         numKeyFields = btree.getComparatorFactories().length;
         deleteTb = new ArrayTupleBuilder(numKeyFields);
     }
-    
+
     @Override
-    public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, TreeIndexException {        
+    public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
         BTree.BTreeAccessor accessor = (BTree.BTreeAccessor) indexAccessor;
         ITreeIndexCursor searchCursor = accessor.createSearchCursor();
         ITreeIndexCursor diskOrderScanCursor = accessor.createDiskOrderScanCursor();
         MultiComparator cmp = accessor.getOpContext().cmp;
         RangePredicate rangePred = new RangePredicate(tuple, tuple, true, true, cmp, cmp);
-        
+
         switch (op) {
             case INSERT:
                 try {
@@ -63,7 +63,7 @@
                     // Ignore duplicate keys, since we get random tuples.
                 }
                 break;
-                
+
             case DELETE:
                 // Create a tuple reference with only key fields.
                 deleteTb.reset();
@@ -77,7 +77,7 @@
                     // Ignore non-existant keys, since we get random tuples.
                 }
                 break;
-                
+
             case UPDATE:
                 try {
                     accessor.update(tuple);
@@ -87,21 +87,21 @@
                     // Ignore not updateable exception due to numKeys == numFields.
                 }
                 break;
-                
+
             case UPSERT:
                 accessor.upsert(tuple);
                 // Upsert should not throw. If it does, there's 
                 // a bigger problem and the test should fail.
                 break;
-                
-            case POINT_SEARCH: 
+
+            case POINT_SEARCH:
                 searchCursor.reset();
                 rangePred.setLowKey(tuple, true);
                 rangePred.setHighKey(tuple, true);
                 accessor.search(searchCursor, rangePred);
                 consumeCursorTuples(searchCursor);
                 break;
-                
+
             case SCAN:
                 searchCursor.reset();
                 rangePred.setLowKey(null, true);
@@ -109,21 +109,21 @@
                 accessor.search(searchCursor, rangePred);
                 consumeCursorTuples(searchCursor);
                 break;
-                
+
             case DISKORDER_SCAN:
                 diskOrderScanCursor.reset();
                 accessor.diskOrderScan(diskOrderScanCursor);
                 consumeCursorTuples(diskOrderScanCursor);
-                break;                            
-            
+                break;
+
             default:
                 throw new HyracksDataException("Op " + op.toString() + " not supported.");
         }
     }
-    
-    private void consumeCursorTuples(ITreeIndexCursor cursor) throws HyracksDataException {
+
+    private void consumeCursorTuples(ITreeIndexCursor cursor) throws HyracksDataException, IndexException {
         try {
-            while(cursor.hasNext()) {
+            while (cursor.hasNext()) {
                 cursor.next();
             }
         } finally {
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 dc4d883..bfde531 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,16 +15,16 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.multithread;
 
-import edu.uci.ics.hyracks.storage.am.common.AbstractTreeIndexTestWorker;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestWorkerFactory;
+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;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
 
-public class BTreeTestWorkerFactory implements ITreeIndexTestWorkerFactory {
+public class BTreeTestWorkerFactory implements IIndexTestWorkerFactory {
     @Override
-    public AbstractTreeIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
-            ITreeIndex index, int numBatches) {
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
+            IIndex index, int numBatches) {
         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/AbstractBTreeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java
index f4eca1b..ef9a456 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java
@@ -23,24 +23,25 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public abstract class AbstractBTreeTest {
-	protected final Logger LOGGER = Logger.getLogger(BTreeTestHarness.class.getName());
-	protected final BTreeTestHarness harness;
-	
-	public AbstractBTreeTest() {
-		harness = new BTreeTestHarness();
+    protected final Logger LOGGER = Logger.getLogger(BTreeTestHarness.class.getName());
+
+    protected final BTreeTestHarness harness;
+
+    public AbstractBTreeTest() {
+        harness = new BTreeTestHarness();
     }
-    
+
     public AbstractBTreeTest(int pageSize, int numPages, int maxOpenFiles, int hyracksFrameSize) {
-    	harness = new BTreeTestHarness(pageSize, numPages, maxOpenFiles, hyracksFrameSize);
+        harness = new BTreeTestHarness(pageSize, numPages, maxOpenFiles, hyracksFrameSize);
     }
-	
-	@Before
-	public void setUp() throws HyracksDataException {
-		harness.setUp();
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        harness.setUp();
     }
-	
-	@After
+
+    @After
     public void tearDown() throws HyracksDataException {
-		harness.tearDown();
+        harness.tearDown();
     }
 }
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 b820f93..1d63b85 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,39 +18,40 @@
 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.io.FileReference;
 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.btree.frames.BTreeLeafFrameType;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 @SuppressWarnings("rawtypes")
 public class BTreeTestContext extends OrderedIndexTestContext {
-    
+
     public BTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
         super(fieldSerdes, treeIndex);
     }
 
     @Override
     public int getKeyFieldCount() {
-        BTree btree = (BTree) treeIndex;
+        BTree btree = (BTree) index;
         return btree.getComparatorFactories().length;
     }
-    
+
     @Override
     public IBinaryComparatorFactory[] getComparatorFactories() {
-        BTree btree = (BTree) treeIndex;
+        BTree btree = (BTree) index;
         return btree.getComparatorFactories();
     }
-    
-    public static BTreeTestContext create(IBufferCache bufferCache, int btreeFileId, ISerializerDeserializer[] fieldSerdes, int numKeyFields, BTreeLeafFrameType leafType) throws Exception {        
+
+    public static BTreeTestContext create(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            FileReference file, ISerializerDeserializer[] fieldSerdes, int numKeyFields, BTreeLeafFrameType leafType)
+            throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
-        BTree btree = BTreeUtils.createBTree(bufferCache, NoOpOperationCallback.INSTANCE, typeTraits, cmpFactories, leafType);
-        btree.create(btreeFileId);
-        btree.open(btreeFileId);
+        BTree btree = BTreeUtils.createBTree(bufferCache, fileMapProvider, typeTraits, cmpFactories, leafType, file);
         BTreeTestContext testCtx = new BTreeTestContext(fieldSerdes, btree);
         return testCtx;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestHarness.java
index 1b450d8..e357bf5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestHarness.java
@@ -24,109 +24,100 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestUtils;
 
-public class BTreeTestHarness {    
+public class BTreeTestHarness {
     public static final BTreeLeafFrameType[] LEAF_FRAMES_TO_TEST = new BTreeLeafFrameType[] {
-        BTreeLeafFrameType.REGULAR_NSM, BTreeLeafFrameType.FIELD_PREFIX_COMPRESSED_NSM };
-    
+            BTreeLeafFrameType.REGULAR_NSM, BTreeLeafFrameType.FIELD_PREFIX_COMPRESSED_NSM };
+
     private static final long RANDOM_SEED = 50;
-    private static final int DEFAULT_PAGE_SIZE = 256;
-    private static final int DEFAULT_NUM_PAGES = 100;
-    private static final int DEFAULT_MAX_OPEN_FILES = 10;
-    private static final int DEFAULT_HYRACKS_FRAME_SIZE = 128;
-    
+
     protected final int pageSize;
     protected final int numPages;
     protected final int maxOpenFiles;
     protected final int hyracksFrameSize;
-        
-    protected IHyracksTaskContext ctx; 
+
+    protected IHyracksTaskContext ctx;
     protected IBufferCache bufferCache;
-    protected int btreeFileId;
-    
+    protected IFileMapProvider fileMapProvider;
+    protected FileReference file;
+
     protected final Random rnd = new Random();
     protected final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
     protected final String tmpDir = System.getProperty("java.io.tmpdir");
     protected final String sep = System.getProperty("file.separator");
     protected String fileName;
-    
+
     public BTreeTestHarness() {
-    	this.pageSize = DEFAULT_PAGE_SIZE;
-    	this.numPages = DEFAULT_NUM_PAGES;
-    	this.maxOpenFiles = DEFAULT_MAX_OPEN_FILES;
-    	this.hyracksFrameSize = DEFAULT_HYRACKS_FRAME_SIZE;
+        this.pageSize = AccessMethodTestsConfig.BTREE_PAGE_SIZE;
+        this.numPages = AccessMethodTestsConfig.BTREE_NUM_PAGES;
+        this.maxOpenFiles = AccessMethodTestsConfig.BTREE_MAX_OPEN_FILES;
+        this.hyracksFrameSize = AccessMethodTestsConfig.BTREE_HYRACKS_FRAME_SIZE;
     }
-    
+
     public BTreeTestHarness(int pageSize, int numPages, int maxOpenFiles, int hyracksFrameSize) {
-    	this.pageSize = pageSize;
-    	this.numPages = numPages;
-    	this.maxOpenFiles = maxOpenFiles;
-    	this.hyracksFrameSize = hyracksFrameSize;
+        this.pageSize = pageSize;
+        this.numPages = numPages;
+        this.maxOpenFiles = maxOpenFiles;
+        this.hyracksFrameSize = hyracksFrameSize;
     }
-    
+
     public void setUp() throws HyracksDataException {
         fileName = tmpDir + sep + simpleDateFormat.format(new Date());
         ctx = TestUtils.create(getHyracksFrameSize());
         TestStorageManagerComponentHolder.init(pageSize, numPages, maxOpenFiles);
         bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
-        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
-        FileReference file = new FileReference(new File(fileName));
-        bufferCache.createFile(file);
-        btreeFileId = fmp.lookupFileId(file);
-        bufferCache.openFile(btreeFileId);
+        fileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        file = new FileReference(new File(fileName));
         rnd.setSeed(RANDOM_SEED);
     }
-    
+
     public void tearDown() throws HyracksDataException {
-        bufferCache.closeFile(btreeFileId);
         bufferCache.close();
-        File f = new File(fileName);
-        f.deleteOnExit();
+        file.delete();
     }
-    
+
     public IHyracksTaskContext getHyracksTaskContext() {
-    	return ctx;
+        return ctx;
     }
-    
+
     public IBufferCache getBufferCache() {
-    	return bufferCache;
+        return bufferCache;
     }
-    
-    public int getBTreeFileId() {
-    	return btreeFileId;
+
+    public IFileMapProvider getFileMapProvider() {
+        return fileMapProvider;
     }
-    
+
+    public FileReference getFileReference() {
+        return file;
+    }
+
     public String getFileName() {
         return fileName;
     }
-    
+
     public Random getRandom() {
-    	return rnd;
+        return rnd;
     }
-    
+
     public int getPageSize() {
         return pageSize;
     }
-    
+
     public int getNumPages() {
         return numPages;
     }
-    
+
     public int getHyracksFrameSize() {
         return hyracksFrameSize;
     }
-    
+
     public int getMaxOpenFiles() {
         return maxOpenFiles;
     }
-    
-    public IOperationCallback getOpCallback() {
-        return NoOpOperationCallback.INSTANCE;
-    }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
deleted file mode 100644
index 59c8c46..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
+++ /dev/null
@@ -1,52 +0,0 @@
-<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>hyracks-storage-am-invertedindex-test</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
-  <name>hyracks-storage-am-invertedindex-test</name>
-
-  <parent>
-    <groupId>edu.uci.ics.hyracks</groupId>
-    <artifactId>hyracks-tests</artifactId>
-    <version>0.2.3-SNAPSHOT</version>
-  </parent>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <version>2.0.2</version>
-        <configuration>
-          <source>1.7</source>
-          <target>1.7</target>
-          <encoding>UTF-8</encoding>
-          <fork>true</fork>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-  <dependencies>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-storage-am-invertedindex</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-test-support</artifactId>
-  		<version>0.2.3-SNAPSHOT</version>
-  		<type>jar</type>
-  		<scope>test</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>junit</groupId>
-  		<artifactId>junit</artifactId>
-  		<version>4.8.1</version>
-  		<type>jar</type>
-  		<scope>test</scope>
-  	</dependency>
-  </dependencies>
-</project>
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
deleted file mode 100644
index e086af6..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Random;
-
-import org.junit.After;
-import org.junit.Before;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-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.io.FileReference;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-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.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-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.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeElementInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.util.InvertedIndexUtils;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestUtils;
-
-public abstract class AbstractInvIndexSearchTest extends AbstractInvIndexTest {
-    protected final int PAGE_SIZE = 32768;
-    protected final int NUM_PAGES = 100;
-    protected final int MAX_OPEN_FILES = 10;
-    protected final int HYRACKS_FRAME_SIZE = 32768;
-    protected IHyracksTaskContext taskCtx = TestUtils.create(HYRACKS_FRAME_SIZE);
-
-    protected IBufferCache bufferCache;
-    protected IFileMapProvider fmp;
-
-    // --- BTREE ---
-
-    // create file refs
-    protected FileReference btreeFile = new FileReference(new File(btreeFileName));
-    protected int btreeFileId;
-
-    // declare token type traits
-    protected ITypeTraits[] tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
-    protected ITypeTraits[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
-
-    // declare btree keys
-    protected int btreeKeyFieldCount = 1;
-    protected IBinaryComparatorFactory[] btreeCmpFactories = new IBinaryComparatorFactory[btreeKeyFieldCount];
-
-    // btree frame factories
-    protected TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(btreeTypeTraits);
-    protected ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
-    protected ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
-    protected ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
-    // btree frames
-    protected ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
-    protected ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
-
-    protected IFreePageManager freePageManager;
-
-    protected BTree btree;
-
-    // --- INVERTED INDEX ---
-
-    protected FileReference invListsFile = new FileReference(new File(invListsFileName));
-    protected int invListsFileId;
-
-    protected int invListFields = 1;
-    protected ITypeTraits[] invListTypeTraits = new ITypeTraits[invListFields];
-
-    protected int invListKeys = 1;
-    protected IBinaryComparatorFactory[] invListCmpFactories = new IBinaryComparatorFactory[invListKeys];
-
-    protected InvertedIndex invIndex;
-
-    protected Random rnd = new Random();
-
-    protected ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
-    protected ArrayTupleReference tuple = new ArrayTupleReference();
-
-    protected ISerializerDeserializer[] insertSerde = { UTF8StringSerializerDeserializer.INSTANCE,
-            IntegerSerializerDeserializer.INSTANCE };
-    protected RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
-
-    protected ArrayList<ArrayList<Integer>> checkInvLists = new ArrayList<ArrayList<Integer>>();
-
-    protected int maxId = 1000000;
-    protected int[] scanCountArray = new int[maxId];
-    protected ArrayList<Integer> expectedResults = new ArrayList<Integer>();
-
-    protected ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
-    protected RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
-
-    protected ArrayTupleBuilder queryTb = new ArrayTupleBuilder(querySerde.length);
-    protected ArrayTupleReference queryTuple = new ArrayTupleReference();
-
-    protected ITokenFactory tokenFactory;
-    protected IBinaryTokenizer tokenizer;
-
-    protected IIndexCursor resultCursor;
-
-    protected abstract void setTokenizer();
-    
-    /**
-     * Initialize members, generate data, and bulk load the inverted index.
-     */
-    @Before
-    public void start() throws Exception {
-        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
-        bufferCache = TestStorageManagerComponentHolder.getBufferCache(taskCtx);
-        fmp = TestStorageManagerComponentHolder.getFileMapProvider(taskCtx);
-
-        // --- BTREE ---
-
-        bufferCache.createFile(btreeFile);
-        btreeFileId = fmp.lookupFileId(btreeFile);
-        bufferCache.openFile(btreeFileId);
-
-        btreeCmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
-
-        btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, btreeTypeTraits.length, btreeCmpFactories,
-                freePageManager, interiorFrameFactory, leafFrameFactory);
-        btree.create(btreeFileId);
-        btree.open(btreeFileId);
-
-        // --- INVERTED INDEX ---
-
-        setTokenizer();
-        
-        bufferCache.createFile(invListsFile);
-        invListsFileId = fmp.lookupFileId(invListsFile);
-        bufferCache.openFile(invListsFileId);
-
-        invListTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
-        invListCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-
-        IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
-        invIndex = new InvertedIndex(bufferCache, btree, invListTypeTraits, invListCmpFactories, invListBuilder);
-        invIndex.open(invListsFileId);
-
-        rnd.setSeed(50);
-    }
-
-    @After
-    public void deinit() throws HyracksDataException {
-        AbstractInvIndexTest.tearDown();
-        btree.close();
-        invIndex.close();
-        bufferCache.closeFile(btreeFileId);
-        bufferCache.closeFile(invListsFileId);
-        bufferCache.close();
-    }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexTest.java
deleted file mode 100644
index cc8ab15..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex;
-
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.logging.Logger;
-
-public abstract class AbstractInvIndexTest {
-
-	protected static final Logger LOGGER = Logger
-			.getLogger(AbstractInvIndexTest.class.getName());
-
-	protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
-			"ddMMyy-hhmmssSS");
-	protected final static String tmpDir = System.getProperty("java.io.tmpdir");
-	protected final static String sep = System.getProperty("file.separator");
-	protected final static String baseFileName = tmpDir + sep
-			+ simpleDateFormat.format(new Date());
-	protected final static String btreeFileName = baseFileName + "btree";
-	protected final static String invListsFileName = baseFileName + "invlists";
-
-	public static void tearDown() {
-		File btreeFile = new File(btreeFileName);
-		btreeFile.deleteOnExit();
-		File invListsFile = new File(invListsFileName);
-		invListsFile.deleteOnExit();
-	}
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
deleted file mode 100644
index 9fdc1c4..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
+++ /dev/null
@@ -1,294 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import junit.framework.Assert;
-
-import org.junit.AfterClass;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-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.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-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.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.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
-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.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-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.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeElementInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeElementInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
-import edu.uci.ics.hyracks.storage.am.invertedindex.util.InvertedIndexUtils;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestUtils;
-
-public class BulkLoadTest extends AbstractInvIndexTest {
-
-    private static final int PAGE_SIZE = 32768;
-    private static final int NUM_PAGES = 100;
-    private static final int MAX_OPEN_FILES = 10;
-    private static final int HYRACKS_FRAME_SIZE = 32768;
-    private IHyracksTaskContext stageletCtx = TestUtils.create(HYRACKS_FRAME_SIZE);
-
-    /**
-     * This test generates a list of <word-token, id> pairs which are pre-sorted
-     * on the token. Those pairs for the input to an inverted-index bulk load.
-     * The contents of the inverted lists are verified against the generated
-     * data.
-     */
-    @Test
-    public void singleFieldPayloadTest() throws Exception {
-
-        TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
-        IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(stageletCtx);
-        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(stageletCtx);
-
-        // create file refs
-        FileReference btreeFile = new FileReference(new File(btreeFileName));
-        bufferCache.createFile(btreeFile);
-        int btreeFileId = fmp.lookupFileId(btreeFile);
-        bufferCache.openFile(btreeFileId);
-
-        FileReference invListsFile = new FileReference(new File(invListsFileName));
-        bufferCache.createFile(invListsFile);
-        int invListsFileId = fmp.lookupFileId(invListsFile);
-        bufferCache.openFile(invListsFileId);
-
-        // Declare token type traits, and compute BTree type traits.
-        ITypeTraits[] tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
-        ITypeTraits[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
-
-        // declare btree keys
-        int keyFieldCount = 1;
-        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
-        cmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
-
-        TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(btreeTypeTraits);
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
-        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
-        ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
-
-        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
-
-        BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, btreeTypeTraits.length, cmpFactories,
-                freePageManager, interiorFrameFactory, leafFrameFactory);
-        btree.create(btreeFileId);
-        btree.open(btreeFileId);
-
-        int invListFields = 1;
-        ITypeTraits[] invListTypeTraits = new ITypeTraits[invListFields];
-        invListTypeTraits[0] = IntegerPointable.TYPE_TRAITS;
-
-        int invListKeys = 1;
-        IBinaryComparatorFactory[] invListCmpFactories = new IBinaryComparatorFactory[invListKeys];
-        invListCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-
-        IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
-        InvertedIndex invIndex = new InvertedIndex(bufferCache, btree, invListTypeTraits, invListCmpFactories, invListBuilder);
-        invIndex.open(invListsFileId);
-
-        Random rnd = new Random();
-        rnd.setSeed(50);
-
-        ByteBuffer frame = stageletCtx.allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(stageletCtx.getFrameSize());
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
-        DataOutput dos = tb.getDataOutput();
-
-        ISerializerDeserializer[] insertSerde = { UTF8StringSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE };
-        RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), insertRecDesc);
-        accessor.reset(frame);
-        FrameTupleReference tuple = new FrameTupleReference();
-
-        List<String> tokens = new ArrayList<String>();
-        tokens.add("compilers");
-        tokens.add("computer");
-        tokens.add("databases");
-        tokens.add("fast");
-        tokens.add("hyracks");
-        tokens.add("major");
-        tokens.add("science");
-        tokens.add("systems");
-        tokens.add("university");
-
-        ArrayList<ArrayList<Integer>> checkListElements = new ArrayList<ArrayList<Integer>>();
-        for (int i = 0; i < tokens.size(); i++) {
-            checkListElements.add(new ArrayList<Integer>());
-        }
-
-        int maxId = 1000000;
-        int addProb = 0;
-        int addProbStep = 10;
-
-        IIndexBulkLoadContext ctx = invIndex.beginBulkLoad(BTree.DEFAULT_FILL_FACTOR);
-
-        for (int i = 0; i < tokens.size(); i++) {
-
-            addProb += addProbStep * (i + 1);
-            for (int j = 0; j < maxId; j++) {
-                if ((Math.abs(rnd.nextInt()) % addProb) == 0) {
-
-                    tb.reset();
-                    UTF8StringSerializerDeserializer.INSTANCE.serialize(tokens.get(i), dos);
-                    tb.addFieldEndOffset();
-                    IntegerSerializerDeserializer.INSTANCE.serialize(j, dos);
-                    tb.addFieldEndOffset();
-
-                    checkListElements.get(i).add(j);
-
-                    appender.reset(frame, true);
-                    appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
-                    tuple.reset(accessor, 0);
-
-                    try {
-                        invIndex.bulkLoadAddTuple(tuple, ctx);
-                    } catch (Exception e) {
-                        e.printStackTrace();
-                    }
-                }
-            }
-        }
-        invIndex.endBulkLoad(ctx);
-
-        // ------- START VERIFICATION -----------
-
-        ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
-        FrameTupleReference searchKey = new FrameTupleReference();
-        MultiComparator btreeCmp = MultiComparator.create(cmpFactories);
-        RangePredicate btreePred = new RangePredicate(searchKey, searchKey, true, true, btreeCmp, btreeCmp);
-
-        IInvertedListCursor invListCursor = new FixedSizeElementInvertedListCursor(bufferCache, invListsFileId,
-                invListTypeTraits);
-
-        ISerializerDeserializer[] tokenSerde = { UTF8StringSerializerDeserializer.INSTANCE };
-        RecordDescriptor tokenRecDesc = new RecordDescriptor(tokenSerde);
-        FrameTupleAppender tokenAppender = new FrameTupleAppender(stageletCtx.getFrameSize());
-        ArrayTupleBuilder tokenTupleBuilder = new ArrayTupleBuilder(1);
-        DataOutput tokenDos = tokenTupleBuilder.getDataOutput();
-        IFrameTupleAccessor tokenAccessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), tokenRecDesc);
-        tokenAccessor.reset(frame);
-
-        ITreeIndexAccessor btreeAccessor = invIndex.getBTree().createAccessor();
-
-        // verify created inverted lists one-by-one
-        for (int i = 0; i < tokens.size(); i++) {
-
-            tokenTupleBuilder.reset();
-            UTF8StringSerializerDeserializer.INSTANCE.serialize(tokens.get(i), tokenDos);
-            tokenTupleBuilder.addFieldEndOffset();
-
-            tokenAppender.reset(frame, true);
-            tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(), tokenTupleBuilder.getByteArray(), 0,
-                    tokenTupleBuilder.getSize());
-
-            searchKey.reset(tokenAccessor, 0);
-
-            invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursor);
-
-            invListCursor.pinPagesSync();
-            int checkIndex = 0;
-            while (invListCursor.hasNext()) {
-                invListCursor.next();
-                ITupleReference invListTuple = invListCursor.getTuple();
-                int invListElement = IntegerSerializerDeserializer.getInt(invListTuple.getFieldData(0),
-                        invListTuple.getFieldStart(0));
-                int checkInvListElement = checkListElements.get(i).get(checkIndex).intValue();
-                Assert.assertEquals(invListElement, checkInvListElement);
-                checkIndex++;
-            }
-            invListCursor.unpinPages();
-            Assert.assertEquals(checkIndex, checkListElements.get(i).size());
-        }
-
-        // check that non-existing tokens have an empty inverted list
-        List<String> nonExistingTokens = new ArrayList<String>();
-        nonExistingTokens.add("watermelon");
-        nonExistingTokens.add("avocado");
-        nonExistingTokens.add("lemon");
-
-        for (int i = 0; i < nonExistingTokens.size(); i++) {
-
-            tokenTupleBuilder.reset();
-            UTF8StringSerializerDeserializer.INSTANCE.serialize(nonExistingTokens.get(i), tokenDos);
-            tokenTupleBuilder.addFieldEndOffset();
-
-            tokenAppender.reset(frame, true);
-            tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(), tokenTupleBuilder.getByteArray(), 0,
-                    tokenTupleBuilder.getSize());
-
-            searchKey.reset(tokenAccessor, 0);
-
-            invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursor);
-
-            invListCursor.pinPagesSync();
-            Assert.assertEquals(invListCursor.hasNext(), false);
-            invListCursor.unpinPages();
-        }
-
-        btree.close();
-        bufferCache.closeFile(btreeFileId);
-        bufferCache.closeFile(invListsFileId);
-        bufferCache.close();
-    }
-
-    @AfterClass
-    public static void deinit() {
-        AbstractInvIndexTest.tearDown();
-    }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java
deleted file mode 100644
index 9c7ec09..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/FixedSizeFrameTupleTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Random;
-
-import junit.framework.Assert;
-
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-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.invertedindex.impls.FixedSizeFrameTupleAccessor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeFrameTupleAppender;
-
-public class FixedSizeFrameTupleTest {
-
-    private static int FRAME_SIZE = 4096;
-
-    private Random rnd = new Random(50);
-
-    /**
-     * This test verifies the correct behavior of the FixedSizeFrameTuple class.
-     * Frames containing FixedSizeFrameTuple's require neither tuple slots nor
-     * field slots. The tests inserts generated data into a frame until the
-     * frame is full, and then verifies the frame's contents.
-     * 
-     */
-    @Test
-    public void singleFieldTest() throws Exception {
-        ByteBuffer buffer = ByteBuffer.allocate(FRAME_SIZE);
-
-        ITypeTraits[] fields = new ITypeTraits[1];
-        fields[0] = IntegerPointable.TYPE_TRAITS;
-
-        FixedSizeFrameTupleAppender ftapp = new FixedSizeFrameTupleAppender(FRAME_SIZE, fields);
-        FixedSizeFrameTupleAccessor ftacc = new FixedSizeFrameTupleAccessor(FRAME_SIZE, fields);
-
-        boolean frameHasSpace = true;
-
-        ArrayList<Integer> check = new ArrayList<Integer>();
-
-        ftapp.reset(buffer, true);
-        while (frameHasSpace) {
-            int val = rnd.nextInt();
-            frameHasSpace = ftapp.append(val);
-            if (frameHasSpace) {
-                check.add(val);
-                ftapp.incrementTupleCount(1);
-            }
-        }
-
-        ftacc.reset(buffer);
-        for (int i = 0; i < ftacc.getTupleCount(); i++) {
-            int val = IntegerSerializerDeserializer.getInt(ftacc.getBuffer().array(), ftacc.getTupleStartOffset(i));
-            Assert.assertEquals(check.get(i).intValue(), val);
-        }
-    }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
deleted file mode 100644
index 3fb6407..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
+++ /dev/null
@@ -1,237 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.AbstractUTF8Token;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8NGramTokenFactory;
-
-public class NGramTokenizerTest {
-
-    private char PRECHAR = '#';
-    private char POSTCHAR = '$';
-
-    private String str = "Jürgen S. Generic's Car";
-    private byte[] inputBuffer;
-
-    private int gramLength = 3;
-
-    private void getExpectedGrams(String s, int gramLength, ArrayList<String> grams, boolean prePost) {
-
-        String tmp = s.toLowerCase();
-        if (prePost) {
-            StringBuilder preBuilder = new StringBuilder();
-            for (int i = 0; i < gramLength - 1; i++) {
-                preBuilder.append(PRECHAR);
-            }
-            String pre = preBuilder.toString();
-
-            StringBuilder postBuilder = new StringBuilder();
-            for (int i = 0; i < gramLength - 1; i++) {
-                postBuilder.append(POSTCHAR);
-            }
-            String post = postBuilder.toString();
-
-            tmp = pre + s.toLowerCase() + post;
-        }
-
-        for (int i = 0; i < tmp.length() - gramLength + 1; i++) {
-            String gram = tmp.substring(i, i + gramLength);
-            grams.add(gram);
-        }
-    }
-
-    @Before
-    public void init() throws Exception {
-        // serialize string into bytes
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        DataOutput dos = new DataOutputStream(baos);
-        dos.writeUTF(str);
-        inputBuffer = baos.toByteArray();
-    }
-
-    void runTestNGramTokenizerWithCountedHashedUTF8Tokens(boolean prePost) throws IOException {
-        HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
-        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, false,
-                false, tokenFactory);
-        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
-
-        ArrayList<String> expectedGrams = new ArrayList<String>();
-        getExpectedGrams(str, gramLength, expectedGrams, prePost);
-        ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
-        HashMap<String, Integer> gramCounts = new HashMap<String, Integer>();
-        for (String s : expectedGrams) {
-            Integer count = gramCounts.get(s);
-            if (count == null) {
-                count = 1;
-                gramCounts.put(s, count);
-            } else {
-                count++;
-            }
-
-            int hash = tokenHash(s, count);
-            expectedHashedGrams.add(hash);
-        }
-
-        int tokenCount = 0;
-
-        while (tokenizer.hasNext()) {
-            tokenizer.next();
-
-            // serialize hashed token
-            GrowableArray tokenStorage = new GrowableArray();
-
-            IToken token = tokenizer.getToken();
-            token.serializeToken(tokenStorage);
-
-            // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
-            DataInput in = new DataInputStream(bais);
-
-            Integer hashedGram = in.readInt();
-
-            // System.out.println(hashedGram);
-
-            Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
-
-            tokenCount++;
-        }
-        // System.out.println("---------");
-    }
-
-    void runTestNGramTokenizerWithHashedUTF8Tokens(boolean prePost) throws IOException {
-        HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
-        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
-                tokenFactory);
-        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
-
-        ArrayList<String> expectedGrams = new ArrayList<String>();
-        getExpectedGrams(str, gramLength, expectedGrams, prePost);
-        ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
-        for (String s : expectedGrams) {
-            int hash = tokenHash(s, 1);
-            expectedHashedGrams.add(hash);
-        }
-
-        int tokenCount = 0;
-
-        while (tokenizer.hasNext()) {
-            tokenizer.next();
-
-            // serialize hashed token
-            GrowableArray tokenStorage = new GrowableArray();
-
-            IToken token = tokenizer.getToken();
-            token.serializeToken(tokenStorage);
-
-            // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
-            DataInput in = new DataInputStream(bais);
-
-            Integer hashedGram = in.readInt();
-
-            // System.out.println(hashedGram);
-
-            Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
-
-            tokenCount++;
-        }
-        // System.out.println("---------");
-    }
-
-    void runTestNGramTokenizerWithUTF8Tokens(boolean prePost) throws IOException {
-        UTF8NGramTokenFactory tokenFactory = new UTF8NGramTokenFactory();
-        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
-                tokenFactory);
-        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
-
-        ArrayList<String> expectedGrams = new ArrayList<String>();
-        getExpectedGrams(str, gramLength, expectedGrams, prePost);
-
-        int tokenCount = 0;
-
-        while (tokenizer.hasNext()) {
-            tokenizer.next();
-
-            // serialize hashed token
-            GrowableArray tokenStorage = new GrowableArray();
-
-            IToken token = tokenizer.getToken();
-            token.serializeToken(tokenStorage);
-
-            // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
-            DataInput in = new DataInputStream(bais);
-
-            String strGram = in.readUTF();
-
-            // System.out.println("\"" + strGram + "\"");
-
-            Assert.assertEquals(expectedGrams.get(tokenCount), strGram);
-
-            tokenCount++;
-        }
-        // System.out.println("---------");
-    }
-
-    @Test
-    public void testNGramTokenizerWithCountedHashedUTF8Tokens() throws Exception {
-        runTestNGramTokenizerWithCountedHashedUTF8Tokens(false);
-        runTestNGramTokenizerWithCountedHashedUTF8Tokens(true);
-    }
-
-    @Test
-    public void testNGramTokenizerWithHashedUTF8Tokens() throws Exception {
-        runTestNGramTokenizerWithHashedUTF8Tokens(false);
-        runTestNGramTokenizerWithHashedUTF8Tokens(true);
-    }
-
-    @Test
-    public void testNGramTokenizerWithUTF8Tokens() throws IOException {
-        runTestNGramTokenizerWithUTF8Tokens(false);
-        runTestNGramTokenizerWithUTF8Tokens(true);
-    }
-
-    public int tokenHash(String token, int tokenCount) {
-        int h = AbstractUTF8Token.GOLDEN_RATIO_32;
-        for (int i = 0; i < token.length(); i++) {
-            h ^= token.charAt(i);
-            h *= AbstractUTF8Token.GOLDEN_RATIO_32;
-        }
-        return h + tokenCount;
-    }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
deleted file mode 100644
index 33bf0c5..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.logging.Level;
-
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
-import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex.InvertedIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndexSearchPredicate;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.OccurrenceThresholdPanicException;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.TOccurrenceSearcher;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.ConjunctiveSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.JaccardSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
-
-/**
- * The purpose of this test is to evaluate the performance of searches against
- * an inverted index. First, we generate random <token, id> pairs sorted on
- * token, which are bulk loaded into an inverted index. Next, we build random
- * queries from a list of predefined tokens in the index, and measure the
- * performance of executing them with different search modifiers. We test the
- * ConjunctiveSearchModifier and the JaccardSearchModifier.
- * 
- */
-public class SearchPerfTest extends AbstractInvIndexSearchTest {
-
-	protected List<String> tokens = new ArrayList<String>();
-
-	@Override
-	protected void setTokenizer() {
-		tokenFactory = new UTF8WordTokenFactory();
-		tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, false,
-				tokenFactory);
-	}
-	
-	@Before
-	public void start() throws Exception {
-		super.start();
-		loadData();
-	}
-
-	public void loadData() throws HyracksDataException, TreeIndexException {
-		tokens.add("compilers");
-		tokens.add("computer");
-		tokens.add("databases");
-		tokens.add("fast");
-		tokens.add("hyracks");
-		tokens.add("major");
-		tokens.add("science");
-		tokens.add("systems");
-		tokens.add("university");
-
-		for (int i = 0; i < tokens.size(); i++) {
-			checkInvLists.add(new ArrayList<Integer>());
-		}
-
-		// for generating length-skewed inverted lists
-		int addProb = 0;
-		int addProbStep = 10;
-
-		IIndexBulkLoadContext ctx = invIndex.beginBulkLoad(BTree.DEFAULT_FILL_FACTOR);
-
-		for (int i = 0; i < tokens.size(); i++) {
-
-			addProb += addProbStep * (i + 1);
-			for (int j = 0; j < maxId; j++) {
-				if ((Math.abs(rnd.nextInt()) % addProb) == 0) {
-					tb.reset();
-					UTF8StringSerializerDeserializer.INSTANCE.serialize(
-							tokens.get(i), tb.getDataOutput());
-					tb.addFieldEndOffset();
-					IntegerSerializerDeserializer.INSTANCE.serialize(j, tb.getDataOutput());
-					tb.addFieldEndOffset();
-					tuple.reset(tb.getFieldEndOffsets(), tb.getByteArray());
-					checkInvLists.get(i).add(j);
-					try {
-						invIndex.bulkLoadAddTuple(tuple, ctx);
-					} catch (Exception e) {
-						e.printStackTrace();
-					}
-				}
-			}
-		}
-		invIndex.endBulkLoad(ctx);
-	}
-
-	/**
-	 * Determine the expected results with the ScanCount algorithm. The
-	 * ScanCount algorithm is very simple, so we can be confident the results
-	 * are correct.
-	 * 
-	 */
-	protected void fillExpectedResults(int[] queryTokenIndexes,
-			int numQueryTokens, int occurrenceThreshold) {
-		// reset scan count array
-		for (int i = 0; i < maxId; i++) {
-			scanCountArray[i] = 0;
-		}
-
-		// count occurrences
-		for (int i = 0; i < numQueryTokens; i++) {
-			ArrayList<Integer> list = checkInvLists.get(queryTokenIndexes[i]);
-			for (int j = 0; j < list.size(); j++) {
-				scanCountArray[list.get(j)]++;
-			}
-		}
-
-		// check threshold
-		expectedResults.clear();
-		for (int i = 0; i < maxId; i++) {
-			if (scanCountArray[i] >= occurrenceThreshold) {
-				expectedResults.add(i);
-			}
-		}
-	}
-
-	/**
-	 * Generates a specified number of queries. Each query consists of a set of
-	 * randomly chosen tokens that are picked from the pre-defined set of
-	 * tokens. We run each query, measure it's time, and verify it's results
-	 * against the results produced by ScanCount, implemented in
-	 * fillExpectedResults().
-	 * 
-	 */
-	private void runQueries(IInvertedIndexSearchModifier searchModifier,
-			int numQueries) throws Exception {
-
-		rnd.setSeed(50);
-
-		InvertedIndexAccessor accessor = (InvertedIndexAccessor) invIndex.createAccessor();
-		InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizer, searchModifier);
-		
-		// generate random queries
-		int[] queryTokenIndexes = new int[tokens.size()];
-		for (int i = 0; i < numQueries; i++) {
-
-			int numQueryTokens = Math.abs(rnd.nextInt() % tokens.size()) + 1;
-			for (int j = 0; j < numQueryTokens; j++) {
-				queryTokenIndexes[j] = Math.abs(rnd.nextInt() % tokens.size());
-			}
-
-			StringBuilder strBuilder = new StringBuilder();
-			for (int j = 0; j < numQueryTokens; j++) {
-				strBuilder.append(tokens.get(queryTokenIndexes[j]));
-				if (j + 1 != numQueryTokens) {
-					strBuilder.append(" ");
-				}
-			}
-
-			String queryString = strBuilder.toString();
-
-			// Serialize query.
-			queryTb.reset();
-			UTF8StringSerializerDeserializer.INSTANCE.serialize(queryString,
-					queryTb.getDataOutput());
-			queryTb.addFieldEndOffset();
-			queryTuple.reset(queryTb.getFieldEndOffsets(), queryTb.getByteArray());
-
-			// Set query tuple in search predicate.
-			searchPred.setQueryTuple(queryTuple);
-			searchPred.setQueryFieldIndex(0);
-			
-			boolean panic = false;
-
-			resultCursor = accessor.createSearchCursor();
-			int repeats = 1;
-			double totalTime = 0;
-			for (int j = 0; j < repeats; j++) {
-				long timeStart = System.currentTimeMillis();
-				try {
-					resultCursor.reset();
-					accessor.search(resultCursor, searchPred);
-				} catch (OccurrenceThresholdPanicException e) {
-					panic = true;
-				}
-				long timeEnd = System.currentTimeMillis();
-				totalTime += timeEnd - timeStart;
-			}
-			double avgTime = totalTime / (double) repeats;
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info(i + ": " + "\"" + queryString + "\": " + avgTime
-						+ "ms");
-			}
-
-			if (!panic) {
-				TOccurrenceSearcher searcher = (TOccurrenceSearcher) accessor.getSearcher();
-				fillExpectedResults(queryTokenIndexes, numQueryTokens,
-						searcher.getOccurrenceThreshold());
-				// verify results
-				int checkIndex = 0;
-				while (resultCursor.hasNext()) {
-					resultCursor.next();
-					ITupleReference resultTuple = resultCursor.getTuple();
-					int id = IntegerSerializerDeserializer.getInt(
-							resultTuple.getFieldData(0),
-							resultTuple.getFieldStart(0));
-					Assert.assertEquals(expectedResults.get(checkIndex)
-							.intValue(), id);
-					checkIndex++;
-				}
-
-				if (expectedResults.size() != checkIndex) {
-					if (LOGGER.isLoggable(Level.INFO)) {
-						LOGGER.info("CHECKING");
-					}
-					StringBuilder expectedStrBuilder = new StringBuilder();
-					for (Integer x : expectedResults) {
-						expectedStrBuilder.append(x + " ");
-					}
-					if (LOGGER.isLoggable(Level.INFO)) {
-						LOGGER.info(expectedStrBuilder.toString());
-					}
-				}
-
-				Assert.assertEquals(expectedResults.size(), checkIndex);
-			}
-		}
-	}
-
-	/**
-	 * Runs 50 random conjunctive search queries to test the
-	 * ConjunctiveSearchModifier.
-	 * 
-	 */
-	@Test
-	public void conjunctiveKeywordQueryTest() throws Exception {
-		IInvertedIndexSearchModifier searchModifier = new ConjunctiveSearchModifier();
-		runQueries(searchModifier, 50);
-	}
-
-	/**
-	 * Runs 50 random jaccard-based search queries with thresholds 1.0, 0.9,
-	 * 0.8, 0.7, 0.6, 0.5. Tests the JaccardSearchModifier.
-	 * 
-	 */
-	@Test
-	public void jaccardKeywordQueryTest() throws Exception {
-		JaccardSearchModifier searchModifier = new JaccardSearchModifier(1.0f);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("JACCARD: " + 1.0f);
-		}
-		searchModifier.setJaccThresh(1.0f);
-		runQueries(searchModifier, 50);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("JACCARD: " + 0.9f);
-		}
-		searchModifier.setJaccThresh(0.9f);
-		runQueries(searchModifier, 50);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("JACCARD: " + 0.8f);
-		}
-		searchModifier.setJaccThresh(0.8f);
-		runQueries(searchModifier, 50);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("JACCARD: " + 0.7f);
-		}
-		searchModifier.setJaccThresh(0.7f);
-		runQueries(searchModifier, 50);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("JACCARD: " + 0.6f);
-		}
-		searchModifier.setJaccThresh(0.6f);
-		runQueries(searchModifier, 50);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("JACCARD: " + 0.5f);
-		}
-		searchModifier.setJaccThresh(0.5f);
-		runQueries(searchModifier, 50);
-	}
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
deleted file mode 100644
index 47a068b..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
+++ /dev/null
@@ -1,308 +0,0 @@
-/*
- * Copyright 2009-2010 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.invertedindex;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.logging.Level;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
-import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex.InvertedIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndexSearchPredicate;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.OccurrenceThresholdPanicException;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.ConjunctiveSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.EditDistanceSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.JaccardSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8NGramTokenFactory;
-
-public class SearchTest extends AbstractInvIndexSearchTest {
-
-	protected List<String> dataStrings = new ArrayList<String>();
-	protected List<String> firstNames = new ArrayList<String>();
-	protected List<String> lastNames = new ArrayList<String>();
-
-	protected IBinaryComparator[] btreeBinCmps;
-	
-	@Override
-	protected void setTokenizer() {
-		tokenFactory = new UTF8NGramTokenFactory();
-		tokenizer = new NGramUTF8StringBinaryTokenizer(3, false, true, false,
-				tokenFactory);
-	}
-	
-	@Before
-	public void start() throws Exception {
-		super.start();
-		btreeBinCmps = new IBinaryComparator[btreeCmpFactories.length];
-		for (int i = 0; i < btreeCmpFactories.length; i++) {
-			btreeBinCmps[i] = btreeCmpFactories[i].createBinaryComparator();
-		}
-		generateDataStrings();
-		loadData();
-	}
-
-	public void generateDataStrings() {
-		firstNames.add("Kathrin");
-		firstNames.add("Cathrin");
-		firstNames.add("Kathryn");
-		firstNames.add("Cathryn");
-		firstNames.add("Kathrine");
-		firstNames.add("Cathrine");
-		firstNames.add("Kathryne");
-		firstNames.add("Cathryne");
-		firstNames.add("Katherin");
-		firstNames.add("Catherin");
-		firstNames.add("Katheryn");
-		firstNames.add("Catheryn");
-		firstNames.add("Katherine");
-		firstNames.add("Catherine");
-		firstNames.add("Katheryne");
-		firstNames.add("Catheryne");
-		firstNames.add("John");
-		firstNames.add("Jack");
-		firstNames.add("Jonathan");
-		firstNames.add("Nathan");
-
-		lastNames.add("Miller");
-		lastNames.add("Myller");
-		lastNames.add("Keller");
-		lastNames.add("Ketler");
-		lastNames.add("Muller");
-		lastNames.add("Fuller");
-		lastNames.add("Smith");
-		lastNames.add("Smyth");
-		lastNames.add("Smithe");
-		lastNames.add("Smythe");
-
-		// Generate all 'firstName lastName' combinations as data strings
-		for (String f : firstNames) {
-			for (String l : lastNames) {
-				dataStrings.add(f + " " + l);
-			}
-		}
-	}
-
-	private class TokenIdPair implements Comparable<TokenIdPair> {
-	    public final GrowableArray tokenStorage = new GrowableArray();
-		public int id;
-
-		TokenIdPair(IToken token, int id) throws IOException {
-			token.serializeToken(tokenStorage);
-			this.id = id;
-		}
-
-		@Override
-		public int compareTo(TokenIdPair o) {
-			int cmp = btreeBinCmps[0].compare(tokenStorage.getByteArray(), 0,
-			        tokenStorage.getByteArray().length, o.tokenStorage.getByteArray(), 0,
-					o.tokenStorage.getByteArray().length);
-			if (cmp == 0) {
-				return id - o.id;
-			} else {
-				return cmp;
-			}
-		}
-	}
-
-	public void loadData() throws IOException, TreeIndexException {
-		List<TokenIdPair> pairs = new ArrayList<TokenIdPair>();
-		// generate pairs for subsequent sorting and bulk-loading
-		int id = 0;
-		for (String s : dataStrings) {
-			ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
-			DataOutputStream dos = new DataOutputStream(baaos);
-			UTF8StringSerializerDeserializer.INSTANCE.serialize(s, dos);
-			tokenizer.reset(baaos.getByteArray(), 0, baaos.size());
-			while (tokenizer.hasNext()) {
-				tokenizer.next();
-				IToken token = tokenizer.getToken();
-				pairs.add(new TokenIdPair(token, id));
-			}
-			++id;
-		}
-		Collections.sort(pairs);
-
-		// bulk load index
-		IIndexBulkLoadContext ctx = invIndex.beginBulkLoad(BTree.DEFAULT_FILL_FACTOR);
-
-		for (TokenIdPair t : pairs) {
-			tb.reset();
-			tb.addField(t.tokenStorage.getByteArray(), 0,
-					t.tokenStorage.getByteArray().length);
-			IntegerSerializerDeserializer.INSTANCE.serialize(t.id, tb.getDataOutput());
-			tb.addFieldEndOffset();
-			tuple.reset(tb.getFieldEndOffsets(), tb.getByteArray());
-
-			try {
-				invIndex.bulkLoadAddTuple(tuple, ctx);
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
-		}
-		invIndex.endBulkLoad(ctx);
-	}
-
-	/**
-	 * Runs a specified number of randomly picked strings from dataStrings as
-	 * queries. We run each query, measure it's time, and print it's results.
-	 * 
-	 */
-	private void runQueries(IInvertedIndexSearchModifier searchModifier,
-			int numQueries) throws Exception {
-
-		rnd.setSeed(50);
-
-		InvertedIndexAccessor accessor = (InvertedIndexAccessor) invIndex.createAccessor();
-		InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizer, searchModifier);
-		
-		for (int i = 0; i < numQueries; i++) {
-
-			int queryIndex = Math.abs(rnd.nextInt() % dataStrings.size());
-			String queryString = dataStrings.get(queryIndex);
-
-			// Serialize query.
-			queryTb.reset();
-			UTF8StringSerializerDeserializer.INSTANCE.serialize(queryString,
-					queryTb.getDataOutput());
-			queryTb.addFieldEndOffset();
-			queryTuple.reset(queryTb.getFieldEndOffsets(), queryTb.getByteArray());
-
-			// Set query tuple in search predicate.
-			searchPred.setQueryTuple(queryTuple);
-			searchPred.setQueryFieldIndex(0);
-			
-			resultCursor = accessor.createSearchCursor();
-			
-			int repeats = 1;
-			double totalTime = 0;
-			for (int j = 0; j < repeats; j++) {
-				long timeStart = System.currentTimeMillis();
-				try {
-					resultCursor.reset();
-					accessor.search(resultCursor, searchPred);
-				} catch (OccurrenceThresholdPanicException e) {
-					// ignore panic queries
-				}
-				long timeEnd = System.currentTimeMillis();
-				totalTime += timeEnd - timeStart;
-			}
-			double avgTime = totalTime / (double) repeats;
-			StringBuilder strBuilder = new StringBuilder();
-			strBuilder.append(i + ": " + "\"" + queryString + "\": " + avgTime
-					+ "ms" + "\n");
-			strBuilder.append("CANDIDATE RESULTS:\n");
-			while (resultCursor.hasNext()) {
-				resultCursor.next();
-				ITupleReference resultTuple = resultCursor.getTuple();
-				int id = IntegerSerializerDeserializer.getInt(
-						resultTuple.getFieldData(0),
-						resultTuple.getFieldStart(0));
-				strBuilder.append(id + " " + dataStrings.get(id));
-				strBuilder.append('\n');
-			}
-			// remove trailing newline
-			strBuilder.deleteCharAt(strBuilder.length() - 1);
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info(strBuilder.toString());
-			}
-		}
-	}
-
-	/**
-	 * Runs 5 random conjunctive search queries to test the
-	 * ConjunctiveSearchModifier.
-	 * 
-	 */
-	@Test
-	public void conjunctiveQueryTest() throws Exception {
-		IInvertedIndexSearchModifier searchModifier = new ConjunctiveSearchModifier();
-		runQueries(searchModifier, 5);
-	}
-
-	/**
-	 * Runs 5 random jaccard-based search queries with thresholds 0.9, 0.8, 0.7.
-	 * Tests the JaccardSearchModifier.
-	 * 
-	 */
-	@Test
-	public void jaccardQueryTest() throws Exception {
-		JaccardSearchModifier searchModifier = new JaccardSearchModifier(1.0f);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("JACCARD: " + 0.9f);
-		}
-		searchModifier.setJaccThresh(0.9f);
-		runQueries(searchModifier, 5);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("JACCARD: " + 0.8f);
-		}
-		searchModifier.setJaccThresh(0.8f);
-		runQueries(searchModifier, 5);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("JACCARD: " + 0.7f);
-		}
-		searchModifier.setJaccThresh(0.7f);
-		runQueries(searchModifier, 5);
-	}
-
-	/**
-	 * Runs 5 random edit-distance based search queries with thresholds 1, 2, 3.
-	 * Tests the EditDistanceSearchModifier.
-	 * 
-	 */
-	@Test
-	public void editDistanceQueryTest() throws Exception {
-		EditDistanceSearchModifier searchModifier = new EditDistanceSearchModifier(
-				3, 0);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("EDIT DISTANCE: " + 1);
-		}
-		searchModifier.setEdThresh(1);
-		runQueries(searchModifier, 5);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("EDIT DISTANCE: " + 2);
-		}
-		searchModifier.setEdThresh(2);
-		runQueries(searchModifier, 5);
-
-		if (LOGGER.isLoggable(Level.INFO)) {
-			LOGGER.info("EDIT DISTANCE: " + 3);
-		}
-		searchModifier.setEdThresh(3);
-		runQueries(searchModifier, 5);
-	}
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
deleted file mode 100644
index 810c5f5..0000000
--- a/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
+++ /dev/null
@@ -1,219 +0,0 @@
-/**
- * Copyright 2010-2011 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 at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on
- * an "AS IS"; BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations under
- * the License.
- * 
- * Author: Alexander Behm <abehm (at) ics.uci.edu>
- */
-
-package edu.uci.ics.hyracks.storage.am.invertedindex;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-
-import junit.framework.Assert;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.AbstractUTF8Token;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
-
-public class WordTokenizerTest {
-
-    private String text = "Hello World, I would like to inform you of the importance of Foo Bar. Yes, Foo Bar. Jürgen.";
-    private byte[] inputBuffer;
-
-    private ArrayList<String> expectedUTF8Tokens = new ArrayList<String>();
-    private ArrayList<Integer> expectedHashedUTF8Tokens = new ArrayList<Integer>();
-    private ArrayList<Integer> expectedCountedHashedUTF8Tokens = new ArrayList<Integer>();
-
-    private boolean isSeparator(char c) {
-        return !(Character.isLetterOrDigit(c) || Character.getType(c) == Character.OTHER_LETTER || Character.getType(c) == Character.OTHER_NUMBER);
-    }
-    
-    private void tokenize(String text, ArrayList<String> tokens) {
-    	String lowerCaseText = text.toLowerCase();
-    	int startIx = 0;
-    	
-    	// Skip separators at beginning of string.
-    	while(isSeparator(lowerCaseText.charAt(startIx))) {
-    		startIx++;
-    	}
-    	while(startIx < lowerCaseText.length()) {
-    		while(startIx < lowerCaseText.length() && isSeparator(lowerCaseText.charAt(startIx))) {
-        	    startIx++;
-        	}
-    		int tokenStart = startIx;
-    		
-    		while(startIx < lowerCaseText.length() && !isSeparator(lowerCaseText.charAt(startIx))) {
-        	    startIx++;
-        	}
-    		int tokenEnd = startIx;
-    		
-    		// Emit token.
-    		String token = lowerCaseText.substring(tokenStart, tokenEnd);
-    		
-    		tokens.add(token);
-    	}
-    }
-    
-    @Before
-    public void init() throws IOException {
-        // serialize text into bytes
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-        DataOutput dos = new DataOutputStream(baos);
-        dos.writeUTF(text);
-        inputBuffer = baos.toByteArray();
-        
-        // init expected string tokens
-        tokenize(text, expectedUTF8Tokens);
-        
-        // hashed tokens ignoring token count
-        for (int i = 0; i < expectedUTF8Tokens.size(); i++) {
-            int hash = tokenHash(expectedUTF8Tokens.get(i), 1);
-            expectedHashedUTF8Tokens.add(hash);
-        }
-
-        // hashed tokens using token count
-        HashMap<String, Integer> tokenCounts = new HashMap<String, Integer>();
-        for (int i = 0; i < expectedUTF8Tokens.size(); i++) {
-            Integer count = tokenCounts.get(expectedUTF8Tokens.get(i));
-            if (count == null) {
-                count = 1;
-                tokenCounts.put(expectedUTF8Tokens.get(i), count);
-            } else {
-                count++;
-            }
-
-            int hash = tokenHash(expectedUTF8Tokens.get(i), count);
-            expectedCountedHashedUTF8Tokens.add(hash);
-        }
-    }
-
-    @Test
-    public void testWordTokenizerWithCountedHashedUTF8Tokens() throws IOException {
-
-        HashedUTF8WordTokenFactory tokenFactory = new HashedUTF8WordTokenFactory();
-        DelimitedUTF8StringBinaryTokenizer tokenizer = new DelimitedUTF8StringBinaryTokenizer(false, false,
-                tokenFactory);
-
-        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
-
-        int tokenCount = 0;
-
-        while (tokenizer.hasNext()) {
-            tokenizer.next();
-
-            // serialize token
-            GrowableArray tokenStorage = new GrowableArray();
-
-            IToken token = tokenizer.getToken();
-            token.serializeToken(tokenStorage);
-
-            // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
-            DataInput in = new DataInputStream(bais);
-
-            Integer hashedToken = in.readInt();
-
-            Assert.assertEquals(hashedToken, expectedCountedHashedUTF8Tokens.get(tokenCount));
-
-            tokenCount++;
-        }
-    }
-
-    @Test
-    public void testWordTokenizerWithHashedUTF8Tokens() throws IOException {
-
-        HashedUTF8WordTokenFactory tokenFactory = new HashedUTF8WordTokenFactory();
-        DelimitedUTF8StringBinaryTokenizer tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, false, tokenFactory);
-
-        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
-
-        int tokenCount = 0;
-
-        while (tokenizer.hasNext()) {
-            tokenizer.next();
-
-            // serialize token
-            GrowableArray tokenStorage = new GrowableArray();
-
-            IToken token = tokenizer.getToken();
-            token.serializeToken(tokenStorage);
-
-            // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
-            DataInput in = new DataInputStream(bais);
-
-            Integer hashedToken = in.readInt();
-
-            Assert.assertEquals(expectedHashedUTF8Tokens.get(tokenCount), hashedToken);
-
-            tokenCount++;
-        }
-    }
-
-    @Test
-    public void testWordTokenizerWithUTF8Tokens() throws IOException {
-
-        UTF8WordTokenFactory tokenFactory = new UTF8WordTokenFactory();
-        DelimitedUTF8StringBinaryTokenizer tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, false, tokenFactory);
-
-        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
-
-        int tokenCount = 0;
-
-        while (tokenizer.hasNext()) {
-            tokenizer.next();
-
-            // serialize hashed token
-            GrowableArray tokenStorage = new GrowableArray();
-
-            IToken token = tokenizer.getToken();
-            token.serializeToken(tokenStorage);
-
-            // deserialize token
-            ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
-            DataInput in = new DataInputStream(bais);
-
-            String strToken = in.readUTF();
-
-            Assert.assertEquals(expectedUTF8Tokens.get(tokenCount), strToken);
-
-            tokenCount++;
-        }
-    }
-
-    // JAQL Hash
-    public int tokenHash(String token, int tokenCount) {
-        int h = AbstractUTF8Token.GOLDEN_RATIO_32;
-        for (int i = 0; i < token.length(); i++) {
-        	h ^= token.charAt(i);
-            h *= AbstractUTF8Token.GOLDEN_RATIO_32;
-        }
-        return h + tokenCount;
-    }
-}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
new file mode 100644
index 0000000..72bfb51
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
@@ -0,0 +1,45 @@
+<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>
+  <artifactId>hyracks-storage-am-lsm-btree-test</artifactId>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-tests</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>  	
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-lsm-btree</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>  	
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-common</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-test-support</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<scope>compile</scope>
+  	</dependency>
+  </dependencies>
+</project>
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
new file mode 100644
index 0000000..4bd1910
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+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.btree.OrderedIndexBulkLoadTest;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+
+@SuppressWarnings("rawtypes")
+public class LSMBTreeBulkLoadTest extends OrderedIndexBulkLoadTest {
+
+    public LSMBTreeBulkLoadTest() {
+        super(LSMBTreeTestHarness.LEAF_FRAMES_TO_TEST, 1);
+    }
+
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+            BTreeLeafFrameType leafType) throws Exception {
+        return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Override
+    protected Random getRandom() {
+        return harness.getRandom();
+    }
+}
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
new file mode 100644
index 0000000..069faad
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+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.btree.OrderedIndexDeleteTest;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+
+@SuppressWarnings("rawtypes")
+public class LSMBTreeDeleteTest extends OrderedIndexDeleteTest {
+
+    public LSMBTreeDeleteTest() {
+        super(LSMBTreeTestHarness.LEAF_FRAMES_TO_TEST);
+    }
+
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+            BTreeLeafFrameType leafType) throws Exception {
+        return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Override
+    protected Random getRandom() {
+        return harness.getRandom();
+    }
+}
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
new file mode 100644
index 0000000..539ed3e
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+import org.junit.After;
+import org.junit.Before;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexExamplesTest;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+
+public class LSMBTreeExamplesTest extends OrderedIndexExamplesTest {
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+
+    @Override
+    protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields) throws TreeIndexException {
+        return LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), typeTraits, cmpFactories, bloomFilterKeyFields,
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+}
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
new file mode 100644
index 0000000..f17e3c8
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+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.btree.OrderedIndexInsertTest;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+
+@SuppressWarnings("rawtypes")
+public class LSMBTreeInsertTest extends OrderedIndexInsertTest {
+
+    public LSMBTreeInsertTest() {
+        super(LSMBTreeTestHarness.LEAF_FRAMES_TO_TEST);
+    }
+
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+            BTreeLeafFrameType leafType) throws Exception {
+        return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Override
+    protected Random getRandom() {
+        return harness.getRandom();
+    }
+}
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
new file mode 100644
index 0000000..24d1f10
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+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;
+import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+
+public class LSMBTreeLifecycleTest extends AbstractIndexLifecycleTest {
+
+    @SuppressWarnings("rawtypes")
+    private final ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE };
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+    private final TreeIndexTestUtils titu = new OrderedIndexTestUtils();
+
+    @SuppressWarnings("rawtypes")
+    private IIndexTestContext<? extends CheckTuple> testCtx;
+
+    @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;
+            }
+        }
+        return true;
+    }
+
+    @Override
+    protected boolean isEmptyIndex() throws Exception {
+        return ((LSMBTree) index).isEmptyIndex();
+    }
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        testCtx = LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+        index = testCtx.getIndex();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        index.deactivate();
+        index.destroy();
+        harness.tearDown();
+    }
+
+    @Override
+    protected void performInsertions() throws Exception {
+        titu.insertIntTuples(testCtx, 10, harness.getRandom());
+    }
+
+    @Override
+    protected void checkInsertions() throws Exception {
+        titu.checkScan(testCtx);
+    }
+
+    @Override
+    protected void clearCheckableInsertions() throws Exception {
+        testCtx.getCheckTuples().clear();
+    }
+
+}
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
new file mode 100644
index 0000000..da36c79
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+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.btree.OrderedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+
+@SuppressWarnings("rawtypes")
+public class LSMBTreeMergeTest extends LSMBTreeMergeTestDriver {
+
+    public LSMBTreeMergeTest() {
+        super(LSMBTreeTestHarness.LEAF_FRAMES_TO_TEST);
+    }
+
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+            BTreeLeafFrameType leafType) throws Exception {
+        return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Override
+    protected Random getRandom() {
+        return harness.getRandom();
+    }
+}
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
new file mode 100644
index 0000000..8b02a8e
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestDriver;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestUtils;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+
+@SuppressWarnings("rawtypes")
+public abstract class LSMBTreeMergeTestDriver extends OrderedIndexTestDriver {
+
+    private final OrderedIndexTestUtils orderedIndexTestUtils;
+
+    public LSMBTreeMergeTestDriver(BTreeLeafFrameType[] leafFrameTypesToTest) {
+        super(leafFrameTypesToTest);
+        this.orderedIndexTestUtils = new OrderedIndexTestUtils();
+    }
+
+    @Override
+    protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, BTreeLeafFrameType leafType,
+            ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
+            throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
+        // Start off with one tree bulk loaded.
+        // 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) {
+            orderedIndexTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+        } else if (fieldSerdes[0] instanceof UTF8StringSerializerDeserializer) {
+            orderedIndexTestUtils.bulkLoadStringTuples(ctx, numTuplesToInsert, getRandom());
+        }
+
+        int maxTreesToMerge = AccessMethodTestsConfig.LSM_BTREE_MAX_TREES_TO_MERGE;
+        for (int i = 0; i < maxTreesToMerge; i++) {
+            for (int j = 0; j < i; j++) {
+                if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
+                    orderedIndexTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+                } else if (fieldSerdes[0] instanceof UTF8StringSerializerDeserializer) {
+                    orderedIndexTestUtils.bulkLoadStringTuples(ctx, numTuplesToInsert, getRandom());
+                }
+            }
+
+            ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
+            accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+
+            orderedIndexTestUtils.checkPointSearches(ctx);
+            orderedIndexTestUtils.checkScan(ctx);
+            orderedIndexTestUtils.checkDiskOrderScan(ctx);
+            orderedIndexTestUtils.checkRangeSearch(ctx, lowKey, highKey, true, true);
+            if (prefixLowKey != null && prefixHighKey != null) {
+                orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
+            }
+        }
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Override
+    protected String getTestOpName() {
+        return "LSM Merge";
+    }
+}
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
new file mode 100644
index 0000000..648e70f
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.AbstractModificationOperationCallbackTest;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
+
+public class LSMBTreeModificationOperationCallbackTest extends AbstractModificationOperationCallbackTest {
+    private static final int NUM_TUPLES = 11;
+
+    private final LSMBTreeTestHarness harness;
+    private final BlockingIOOperationCallbackWrapper ioOpCallback;
+
+    public LSMBTreeModificationOperationCallbackTest() {
+        super();
+        this.ioOpCallback = new BlockingIOOperationCallbackWrapper(NoOpIOOperationCallback.INSTANCE);
+        harness = new LSMBTreeTestHarness();
+    }
+
+    @Override
+    protected void createIndexInstance() throws Exception {
+        index = LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), SerdeUtils.serdesToTypeTraits(keySerdes),
+                SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
+                harness.getMergePolicy(), NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        super.setup();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        harness.tearDown();
+    }
+
+    @Test
+    public void modificationCallbackTest() throws Exception {
+        ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(cb, NoOpOperationCallback.INSTANCE);
+
+        for (int j = 0; j < 2; j++) {
+            isFoundNull = true;
+            for (int i = 0; i < NUM_TUPLES; i++) {
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                accessor.insert(tuple);
+            }
+
+            if (j == 1) {
+                accessor.scheduleFlush(ioOpCallback);
+                ioOpCallback.waitForIO();
+                isFoundNull = true;
+            } else {
+                isFoundNull = false;
+            }
+
+            for (int i = 0; i < NUM_TUPLES; i++) {
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                accessor.upsert(tuple);
+            }
+
+            if (j == 1) {
+                accessor.scheduleFlush(ioOpCallback);
+                ioOpCallback.waitForIO();
+                isFoundNull = true;
+            } else {
+                isFoundNull = false;
+            }
+
+            for (int i = 0; i < NUM_TUPLES; i++) {
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                accessor.delete(tuple);
+            }
+
+            accessor.scheduleFlush(ioOpCallback);
+            ioOpCallback.waitForIO();
+        }
+    }
+}
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
new file mode 100644
index 0000000..3a99c16
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+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.btree.OrderedIndexBulkLoadTest;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+
+@SuppressWarnings("rawtypes")
+public class LSMBTreeMultiBulkLoadTest extends OrderedIndexBulkLoadTest {
+    public LSMBTreeMultiBulkLoadTest() {
+        // Using 5 bulk load rounds.
+        super(LSMBTreeTestHarness.LEAF_FRAMES_TO_TEST, AccessMethodTestsConfig.LSM_BTREE_BULKLOAD_ROUNDS);
+    }
+
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+            BTreeLeafFrameType leafType) throws Exception {
+        return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Override
+    protected Random getRandom() {
+        return harness.getRandom();
+    }
+}
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
new file mode 100644
index 0000000..389c87f
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -0,0 +1,263 @@
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+import java.util.HashSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.AbstractSearchOperationCallbackTest;
+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.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
+
+public class LSMBTreeSearchOperationCallbackTest extends AbstractSearchOperationCallbackTest {
+    private final LSMBTreeTestHarness harness;
+    private final HashSet<Integer> deleteSet;
+
+    public LSMBTreeSearchOperationCallbackTest() {
+        harness = new LSMBTreeTestHarness();
+        deleteSet = new HashSet<Integer>();
+    }
+
+    @Override
+    protected void createIndexInstance() throws Exception {
+        index = LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), SerdeUtils.serdesToTypeTraits(keySerdes),
+                SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
+                harness.getMergePolicy(), NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        super.setup();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        harness.tearDown();
+    }
+
+    @Test
+    public void searchCallbackTest() throws Exception {
+        Future<Boolean> insertFuture = executor.submit(new InsertionTask());
+        Future<Boolean> searchFuture = executor.submit(new SearchTask());
+        Assert.assertTrue(searchFuture.get());
+        Assert.assertTrue(insertFuture.get());
+    }
+
+    private class SearchTask implements Callable<Boolean> {
+        private final ISearchOperationCallback cb;
+        private final IIndexAccessor accessor;
+        private final IIndexCursor cursor;
+        private final RangePredicate predicate;
+        private final ArrayTupleBuilder builder;
+        private final ArrayTupleReference tuple;
+        private final ArrayTupleBuilder expectedTupleToBeLockedBuilder;
+        private final ArrayTupleReference expectedTupleToBeLocked;
+        private final ArrayTupleBuilder expectedTupleToBeCanceledBuilder;
+        private final ArrayTupleReference expectedTupleToBeCanceled;
+
+        private boolean blockOnHigh;
+        private int expectedAfterBlock;
+        private int expectedTupleToBeLockedValue;
+
+        public SearchTask() {
+            this.cb = new SynchronizingSearchOperationCallback();
+            this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb);
+            this.cursor = accessor.createSearchCursor();
+            this.predicate = new RangePredicate();
+            this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+            this.tuple = new ArrayTupleReference();
+            this.expectedTupleToBeLockedBuilder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+            this.expectedTupleToBeLocked = new ArrayTupleReference();
+            this.expectedTupleToBeCanceledBuilder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+            this.expectedTupleToBeCanceled = new ArrayTupleReference();
+
+            this.blockOnHigh = false;
+            this.expectedAfterBlock = -1;
+            this.expectedTupleToBeLockedValue = -1;
+        }
+
+        @Override
+        public Boolean call() throws Exception {
+            lock.lock();
+            try {
+                if (!insertTaskStarted) {
+                    condition.await();
+                }
+
+                // begin a search on [50, +inf), blocking on 75
+                TupleUtils.createIntegerTuple(builder, tuple, 50);
+                predicate.setLowKey(tuple, true);
+                predicate.setHighKey(null, true);
+                accessor.search(cursor, predicate);
+                expectedTupleToBeLockedValue = 50;
+                TupleUtils.createIntegerTuple(builder, expectedTupleToBeLocked, expectedTupleToBeLockedValue);
+                consumeIntTupleRange(50, 75, true, 76);
+
+                // consume tuples [77, 150], blocking on 151
+                consumeIntTupleRange(77, 150, true, 150);
+
+                // consume tuples [152, 300]
+                consumeIntTupleRange(152, 300, false, -1);
+
+                cursor.close();
+            } finally {
+                lock.unlock();
+            }
+
+            return true;
+        }
+
+        private void consumeIntTupleRange(int begin, int end, boolean blockOnHigh, int expectedAfterBlock)
+                throws Exception {
+            if (end < begin) {
+                throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
+            }
+
+            for (int i = begin; i <= end; i++) {
+                if (blockOnHigh == true && i == end) {
+                    this.blockOnHigh = true;
+                    this.expectedAfterBlock = expectedAfterBlock;
+                }
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                if (!cursor.hasNext()) {
+                    Assert.fail("Failed to consume entire tuple range since cursor is exhausted.");
+                }
+                cursor.next();
+                Assert.assertEquals(0, cmp.compare(tuple, cursor.getTuple()));
+            }
+        }
+
+        private class SynchronizingSearchOperationCallback implements ISearchOperationCallback {
+
+            @Override
+            public boolean proceed(ITupleReference tuple) {
+                Assert.assertEquals(0, cmp.compare(SearchTask.this.expectedTupleToBeLocked, tuple));
+                return false;
+            }
+
+            @Override
+            public void reconcile(ITupleReference tuple) throws HyracksDataException {
+                Assert.assertEquals(0, cmp.compare(SearchTask.this.expectedTupleToBeLocked, tuple));
+                if (blockOnHigh) {
+                    TupleUtils.createIntegerTuple(builder, SearchTask.this.tuple, expectedAfterBlock);
+                    condition.signal();
+                    condition.awaitUninterruptibly();
+                    blockOnHigh = false;
+                }
+                expectedTupleToBeLockedValue++;
+                TupleUtils.createIntegerTuple(expectedTupleToBeLockedBuilder, expectedTupleToBeLocked,
+                        expectedTupleToBeLockedValue);
+
+            }
+
+            @Override
+            public void cancel(ITupleReference tuple) throws HyracksDataException {
+                boolean found = false;
+                for (int i : deleteSet) {
+                    TupleUtils.createIntegerTuple(expectedTupleToBeCanceledBuilder, expectedTupleToBeCanceled, i);
+                    if (cmp.compare(SearchTask.this.expectedTupleToBeCanceled, tuple) == 0) {
+                        found = true;
+                        break;
+                    }
+                }
+                Assert.assertTrue(found);
+            }
+
+        }
+    }
+
+    private class InsertionTask implements Callable<Boolean> {
+        private final IIndexAccessor accessor;
+        private final ArrayTupleBuilder builder;
+        private final ArrayTupleReference tuple;
+
+        public InsertionTask() {
+            this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+            this.tuple = new ArrayTupleReference();
+        }
+
+        @Override
+        public Boolean call() throws Exception {
+            lock.lock();
+            try {
+                insertTaskStarted = true;
+
+                // bulkload [101, 150] & [151, 200] as two separate disk components 
+                // insert [50, 100] & [301, 350] to the in-memory component
+                // delete tuple 151
+                bulkloadIntTupleRange(101, 150);
+                bulkloadIntTupleRange(151, 200);
+                insertIntTupleRange(50, 100);
+                insertIntTupleRange(301, 350);
+                int tupleTobeDeletedValue = 151;
+                deleteSet.add(tupleTobeDeletedValue);
+                TupleUtils.createIntegerTuple(builder, tuple, tupleTobeDeletedValue);
+                accessor.delete(tuple);
+                condition.signal();
+                condition.await();
+
+                // delete tuple 75
+                tupleTobeDeletedValue = 75;
+                deleteSet.add(tupleTobeDeletedValue);
+                TupleUtils.createIntegerTuple(builder, tuple, tupleTobeDeletedValue);
+                accessor.delete(tuple);
+                condition.signal();
+                condition.await();
+
+                // insert tuples [201, 300] and delete tuple 151
+                insertIntTupleRange(201, 300);
+                condition.signal();
+            } finally {
+                lock.unlock();
+            }
+
+            return true;
+        }
+
+        private void insertIntTupleRange(int begin, int end) throws Exception {
+            if (end < begin) {
+                throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
+            }
+
+            for (int i = begin; i <= end; i++) {
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                accessor.insert(tuple);
+            }
+        }
+
+        private void bulkloadIntTupleRange(int begin, int end) throws Exception {
+            if (end < begin) {
+                throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
+            }
+
+            IIndexBulkLoader bulkloader = index.createBulkLoader(1.0f, false, end - begin);
+            for (int i = begin; i <= end; i++) {
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                bulkloader.add(tuple);
+            }
+            bulkloader.end();
+        }
+
+    }
+}
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
new file mode 100644
index 0000000..ca89512
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree;
+
+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.btree.OrderedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexUpdateTest;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+
+@SuppressWarnings("rawtypes")
+public class LSMBTreeUpdateTest extends OrderedIndexUpdateTest {
+
+    public LSMBTreeUpdateTest() {
+        super(LSMBTreeTestHarness.LEAF_FRAMES_TO_TEST);
+    }
+
+    private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+            BTreeLeafFrameType leafType) throws Exception {
+        return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Override
+    protected Random getRandom() {
+        return harness.getRandom();
+    }
+}
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
new file mode 100644
index 0000000..c494448
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.multithread;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexMultiThreadTest;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.ProbabilityHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+
+public class LSMBTreeMultiThreadTest extends OrderedIndexMultiThreadTest {
+
+    private LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+
+    private LSMBTreeTestWorkerFactory workerFactory = new LSMBTreeTestWorkerFactory();
+
+    @Override
+    protected void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @Override
+    protected void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected ITreeIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields) throws TreeIndexException {
+        return LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), typeTraits, cmpFactories, bloomFilterKeyFields,
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @Override
+    protected IIndexTestWorkerFactory getWorkerFactory() {
+        return workerFactory;
+    }
+
+    @Override
+    protected ArrayList<TestWorkloadConf> getTestWorkloadConf() {
+        ArrayList<TestWorkloadConf> workloadConfs = new ArrayList<TestWorkloadConf>();
+
+        // Insert only workload.
+        TestOperation[] insertOnlyOps = new TestOperation[] { TestOperation.INSERT };
+        workloadConfs.add(new TestWorkloadConf(insertOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertOnlyOps.length)));
+
+        // Insert and merge workload.
+        TestOperation[] insertMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(insertMergeOps, ProbabilityHelper
+                .getUniformProbDist(insertMergeOps.length)));
+
+        // Inserts mixed with point searches and scans.
+        TestOperation[] insertSearchOnlyOps = new TestOperation[] { TestOperation.INSERT, TestOperation.POINT_SEARCH,
+                TestOperation.SCAN };
+        workloadConfs.add(new TestWorkloadConf(insertSearchOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertSearchOnlyOps.length)));
+
+        // Inserts, updates, and deletes.
+        TestOperation[] insertDeleteUpdateOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.UPDATE };
+        workloadConfs.add(new TestWorkloadConf(insertDeleteUpdateOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteUpdateOps.length)));
+
+        // Inserts, updates, deletes and merges.
+        TestOperation[] insertDeleteUpdateMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.UPDATE, TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(insertDeleteUpdateMergeOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteUpdateMergeOps.length)));
+
+        // All operations except merge.
+        TestOperation[] allNoMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.UPDATE, TestOperation.POINT_SEARCH, TestOperation.SCAN };
+        workloadConfs.add(new TestWorkloadConf(allNoMergeOps, ProbabilityHelper
+                .getUniformProbDist(allNoMergeOps.length)));
+
+        // All operations.
+        TestOperation[] allOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.UPDATE, TestOperation.POINT_SEARCH, TestOperation.SCAN, TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(allOps, ProbabilityHelper.getUniformProbDist(allOps.length)));
+
+        return workloadConfs;
+    }
+
+    @Override
+    protected String getIndexTypeName() {
+        return "LSMBTree";
+    }
+}
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
new file mode 100644
index 0000000..c008f90
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.multithread;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeNonExistentKeyException;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeNotUpdateableException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree.LSMBTreeAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+
+public class LSMBTreeTestWorker extends AbstractIndexTestWorker {
+    private final LSMBTree lsmBTree;
+    private final int numKeyFields;
+    private final ArrayTupleBuilder deleteTb;
+    private final ArrayTupleReference deleteTuple = new ArrayTupleReference();
+
+    public LSMBTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+        super(dataGen, opSelector, index, numBatches);
+        lsmBTree = (LSMBTree) index;
+        numKeyFields = lsmBTree.getComparatorFactories().length;
+        deleteTb = new ArrayTupleBuilder(numKeyFields);
+    }
+
+    @Override
+    public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
+        LSMBTreeAccessor accessor = (LSMBTreeAccessor) indexAccessor;
+        IIndexCursor searchCursor = accessor.createSearchCursor();
+        MultiComparator cmp = accessor.getMultiComparator();
+        RangePredicate rangePred = new RangePredicate(tuple, tuple, true, true, cmp, cmp);
+
+        switch (op) {
+            case INSERT:
+                try {
+                    accessor.insert(tuple);
+                } catch (BTreeDuplicateKeyException e) {
+                    // Ignore duplicate keys, since we get random tuples.
+                }
+                break;
+
+            case DELETE:
+                // Create a tuple reference with only key fields.
+                deleteTb.reset();
+                for (int i = 0; i < numKeyFields; i++) {
+                    deleteTb.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+                }
+                deleteTuple.reset(deleteTb.getFieldEndOffsets(), deleteTb.getByteArray());
+                try {
+                    accessor.delete(deleteTuple);
+                } catch (BTreeNonExistentKeyException e) {
+                    // Ignore non-existant keys, since we get random tuples.
+                }
+                break;
+
+            case UPDATE:
+                try {
+                    accessor.update(tuple);
+                } catch (BTreeNonExistentKeyException e) {
+                    // Ignore non-existant keys, since we get random tuples.
+                } catch (BTreeNotUpdateableException e) {
+                    // Ignore not updateable exception due to numKeys == numFields.
+                }
+                break;
+
+            case POINT_SEARCH:
+                searchCursor.reset();
+                rangePred.setLowKey(tuple, true);
+                rangePred.setHighKey(tuple, true);
+                accessor.search(searchCursor, rangePred);
+                consumeCursorTuples(searchCursor);
+                break;
+
+            case SCAN:
+                searchCursor.reset();
+                rangePred.setLowKey(null, true);
+                rangePred.setHighKey(null, true);
+                accessor.search(searchCursor, rangePred);
+                consumeCursorTuples(searchCursor);
+                break;
+
+            case MERGE:
+                accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+                break;
+
+            default:
+                throw new HyracksDataException("Op " + op.toString() + " not supported.");
+        }
+    }
+}
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
new file mode 100644
index 0000000..03463e6
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorkerFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.multithread;
+
+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;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+
+public class LSMBTreeTestWorkerFactory implements IIndexTestWorkerFactory {
+    @Override
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
+            IIndex index, int numBatches) {
+        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
new file mode 100644
index 0000000..69e2b58
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeBulkLoadRunner.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.perf;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
+
+public class BTreeBulkLoadRunner extends BTreeRunner {
+
+    protected final float fillFactor;
+
+    public BTreeBulkLoadRunner(int numBatches, int pageSize, int numPages, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories, float fillFactor) throws HyracksDataException, BTreeException {
+        super(numBatches, pageSize, numPages, typeTraits, cmpFactories);
+        this.fillFactor = fillFactor;
+    }
+
+    @Override
+    public long runExperiment(DataGenThread dataGen, int numThreads) throws Exception {
+        btree.create();
+        long start = System.currentTimeMillis();
+        IIndexBulkLoader bulkLoader = btree.createBulkLoader(1.0f, false, 0L);
+        for (int i = 0; i < numBatches; i++) {
+            TupleBatch batch = dataGen.tupleBatchQueue.take();
+            for (int j = 0; j < batch.size(); j++) {
+                bulkLoader.add(batch.get(j));
+            }
+        }
+        bulkLoader.end();
+        long end = System.currentTimeMillis();
+        long time = end - start;
+        return time;
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java
new file mode 100644
index 0000000..7e0514b
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.perf;
+
+import java.util.Enumeration;
+import java.util.logging.Level;
+import java.util.logging.LogManager;
+import java.util.logging.Logger;
+
+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.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+
+public class BTreePageSizePerf {
+    public static void main(String[] args) throws Exception {
+        // Disable logging so we can better see the output times.
+        Enumeration<String> loggers = LogManager.getLogManager().getLoggerNames();
+        while(loggers.hasMoreElements()) {
+            String loggerName = loggers.nextElement();
+            Logger logger = LogManager.getLogManager().getLogger(loggerName);
+            logger.setLevel(Level.OFF);
+        }
+        
+        int numTuples = 1000000;
+        int batchSize = 10000;
+        int numBatches = numTuples / batchSize;
+        
+        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE };
+        ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes, 30);
+        
+        IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, fieldSerdes.length);
+        
+        runExperiment(numBatches, batchSize, 1024, 100000, fieldSerdes, cmpFactories, typeTraits);
+        runExperiment(numBatches, batchSize, 2048, 100000, fieldSerdes, cmpFactories, typeTraits);
+        runExperiment(numBatches, batchSize, 4096, 25000, fieldSerdes, cmpFactories, typeTraits);
+        runExperiment(numBatches, batchSize, 8192, 12500, fieldSerdes, cmpFactories, typeTraits);
+        runExperiment(numBatches, batchSize, 16384, 6250, fieldSerdes, cmpFactories, typeTraits);
+        runExperiment(numBatches, batchSize, 32768, 3125, fieldSerdes, cmpFactories, typeTraits);
+        runExperiment(numBatches, batchSize, 65536, 1564, fieldSerdes, cmpFactories, typeTraits);
+        runExperiment(numBatches, batchSize, 131072, 782, fieldSerdes, cmpFactories, typeTraits);
+        runExperiment(numBatches, batchSize, 262144, 391, fieldSerdes, cmpFactories, typeTraits);
+    }
+    
+    private static void runExperiment(int numBatches, int batchSize, int pageSize, int numPages, ISerializerDeserializer[] fieldSerdes, IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] typeTraits) throws Exception {
+        System.out.println("PAGE SIZE: " + pageSize);
+        System.out.println("NUM PAGES: " + numPages);
+        System.out.println("MEMORY: " + (pageSize * numPages));
+        int repeats = 5;
+        long[] times = new long[repeats];
+        //BTreeRunner runner = new BTreeRunner(numTuples, pageSize, numPages, typeTraits, cmp);
+        InMemoryBTreeRunner runner = new InMemoryBTreeRunner(numBatches, pageSize, numPages, typeTraits, cmpFactories);
+        runner.init();
+        int numThreads = 1;
+        for (int i = 0; i < repeats; i++) {
+            DataGenThread dataGen = new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, 30, 50, 10, false);
+            dataGen.start();            
+            times[i] = runner.runExperiment(dataGen, numThreads);
+            System.out.println("TIME " + i + ": " + times[i] + "ms");
+        }
+        runner.deinit();
+        long avgTime = 0;
+        for (int i = 0; i < repeats; i++) {
+            avgTime += times[i];
+        }
+        avgTime /= repeats;
+        System.out.println("AVG TIME: " + avgTime + "ms");
+        System.out.println("-------------------------------");
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeRunner.java
new file mode 100644
index 0000000..8658919
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreeRunner.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.perf;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public class BTreeRunner extends InMemoryBTreeRunner {
+    protected static final int MAX_OPEN_FILES = 10;
+    protected static final int HYRACKS_FRAME_SIZE = 128;
+
+    public BTreeRunner(int numTuples, int pageSize, int numPages, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories) throws HyracksDataException, BTreeException {
+        super(numTuples, pageSize, numPages, typeTraits, cmpFactories);
+    }
+
+    @Override
+    protected void init(int pageSize, int numPages, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories)
+            throws HyracksDataException, BTreeException {
+        IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+        TestStorageManagerComponentHolder.init(pageSize, numPages, MAX_OPEN_FILES);
+        bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        btree = BTreeUtils
+                .createBTree(bufferCache, fmp, typeTraits, cmpFactories, BTreeLeafFrameType.REGULAR_NSM, file);
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/ConcurrentSkipListRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/ConcurrentSkipListRunner.java
new file mode 100644
index 0000000..8f44966
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/ConcurrentSkipListRunner.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.perf;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+
+public class ConcurrentSkipListRunner implements IExperimentRunner {
+    public class TupleComparator implements Comparator<ITupleReference> {
+        private final MultiComparator cmp;
+
+        public TupleComparator(MultiComparator cmp) {
+            this.cmp = cmp;
+        }
+
+        @Override
+        public int compare(ITupleReference o1, ITupleReference o2) {
+            return cmp.compare(o1, o2);
+        }
+    }
+    
+    private final TupleComparator tupleCmp;
+    private final int numBatches;
+    private final int batchSize;
+    private final int tupleSize;
+    private final ITypeTraits[] typeTraits;
+    
+    public ConcurrentSkipListRunner(int numBatches, int batchSize, int tupleSize, ITypeTraits[] typeTraits, MultiComparator cmp) {
+        this.numBatches = numBatches;
+        this.tupleSize = tupleSize;
+        this.batchSize = batchSize;
+        this.typeTraits = typeTraits;
+        tupleCmp = new TupleComparator(cmp);
+    }
+    
+    @Override
+    public long runExperiment(DataGenThread dataGen, int numThreads) throws InterruptedException {
+        ConcurrentSkipListSet<ITupleReference> skipList = new ConcurrentSkipListSet<ITupleReference>(tupleCmp);
+        SkipListThread[] threads = new SkipListThread[numThreads];
+        int threadNumBatches = numBatches / numThreads;
+        for (int i = 0; i < numThreads; i++) {
+            threads[i] = new SkipListThread(dataGen, skipList, threadNumBatches, batchSize);            
+        }
+        // Wait until the tupleBatchQueue is completely full.
+        while (dataGen.tupleBatchQueue.remainingCapacity() != 0) {
+            Thread.sleep(10);
+        }
+        
+        long start = System.currentTimeMillis();
+        for (int i = 0; i < numThreads; i++) {
+            threads[i].start();
+        }
+        for (int i = 0; i < numThreads; i++) {
+            threads[i].join();
+        }
+        long end = System.currentTimeMillis();
+        long time = end - start;
+        return time;
+    }
+
+    @Override
+    public void init() throws Exception {
+    }
+
+    @Override
+    public void deinit() throws Exception {
+    }
+    
+    public void reset() throws Exception {
+    }
+    
+    public class SkipListThread extends Thread {
+    	private final DataGenThread dataGen;
+    	private final ConcurrentSkipListSet<ITupleReference> skipList;
+    	private final int numBatches;
+        public final TypeAwareTupleWriterFactory tupleWriterFactory;
+        public final TypeAwareTupleWriter tupleWriter;
+        public final TypeAwareTupleReference[] tuples;        
+        public final ByteBuffer tupleBuf; 
+
+        public SkipListThread(DataGenThread dataGen, ConcurrentSkipListSet<ITupleReference> skipList, int numBatches, int batchSize) {
+            this.dataGen = dataGen;
+            this.numBatches = numBatches;
+            this.skipList = skipList;
+            tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+            tupleWriter = (TypeAwareTupleWriter) tupleWriterFactory.createTupleWriter();
+            int numTuples = numBatches * batchSize;
+            tuples = new TypeAwareTupleReference[numTuples];
+            tupleBuf = ByteBuffer.allocate(numTuples * tupleSize);
+            for (int i = 0; i < numTuples; i++) {
+                tuples[i] = (TypeAwareTupleReference) tupleWriter.createTupleReference();
+            }
+        }
+    	
+        @Override
+        public void run() {
+            int tupleIndex = 0;
+            try {                
+                for (int i = 0; i < numBatches; i++) {
+                    TupleBatch batch = dataGen.tupleBatchQueue.take();
+                    for (int j = 0; j < batch.size(); j++) {
+                        // Copy the tuple to the buffer and set the pre-created tuple ref.                        
+                        tupleWriter.writeTuple(batch.get(j), tupleBuf.array(), tupleIndex * tupleSize);
+                        tuples[tupleIndex].resetByTupleOffset(tupleBuf, tupleIndex * tupleSize);
+                        skipList.add(tuples[tupleIndex]);
+                        tupleIndex++;
+                    }
+                }
+            } catch (Exception e) {
+                System.out.println(tupleIndex);
+                e.printStackTrace();
+            }
+        }
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/IExperimentRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/IExperimentRunner.java
new file mode 100644
index 0000000..0ea3a71
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/IExperimentRunner.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.perf;
+
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+
+public interface IExperimentRunner {
+    public static int DEFAULT_MAX_OUTSTANDING = 100000;
+    
+    public void init() throws Exception;
+    
+    public long runExperiment(DataGenThread dataGen, int numThreads) throws Exception;
+    
+    public void reset() throws Exception;
+    
+    public void deinit() throws Exception;
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java
new file mode 100644
index 0000000..1b453b7
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemoryBTreeRunner.java
@@ -0,0 +1,146 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.perf;
+
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
+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.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICacheMemoryAllocator;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+
+public class InMemoryBTreeRunner extends Thread implements IExperimentRunner {
+    protected IBufferCache bufferCache;
+    protected FileReference file;
+
+    protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    protected final static String tmpDir = System.getProperty("java.io.tmpdir");
+    protected final static String sep = System.getProperty("file.separator");
+    protected String fileName;
+
+    protected final int numBatches;
+    protected BTree btree;
+
+    public InMemoryBTreeRunner(int numBatches, int pageSize, int numPages, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories) throws HyracksDataException, BTreeException {
+        this.numBatches = numBatches;
+        fileName = tmpDir + sep + simpleDateFormat.format(new Date());
+        file = new FileReference(new File(fileName));
+        init(pageSize, numPages, typeTraits, cmpFactories);
+    }
+
+    protected void init(int pageSize, int numPages, ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories)
+            throws HyracksDataException, BTreeException {
+        ICacheMemoryAllocator allocator = new HeapBufferAllocator();
+        bufferCache = new InMemoryBufferCache(allocator, pageSize, numPages, new TransientFileMapManager());
+        TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        IFreePageManager freePageManager = new InMemoryFreePageManager(bufferCache.getNumPages(), metaFrameFactory);
+        btree = new BTree(bufferCache, new TransientFileMapManager(), freePageManager, interiorFrameFactory,
+                leafFrameFactory, cmpFactories, typeTraits.length, file);
+    }
+
+    @Override
+    public long runExperiment(DataGenThread dataGen, int numThreads) throws Exception {
+        BTreeThread[] threads = new BTreeThread[numThreads];
+        int threadNumBatches = numBatches / numThreads;
+        for (int i = 0; i < numThreads; i++) {
+            threads[i] = new BTreeThread(dataGen, btree, threadNumBatches);
+        }
+        // Wait until the tupleBatchQueue is completely full.
+        while (dataGen.tupleBatchQueue.remainingCapacity() != 0) {
+            Thread.sleep(10);
+        }
+
+        long start = System.currentTimeMillis();
+        for (int i = 0; i < numThreads; i++) {
+            threads[i].start();
+        }
+        for (int i = 0; i < numThreads; i++) {
+            threads[i].join();
+        }
+        long end = System.currentTimeMillis();
+        long time = end - start;
+        return time;
+    }
+
+    @Override
+    public void init() throws Exception {
+    }
+
+    @Override
+    public void deinit() throws Exception {
+        bufferCache.close();
+    }
+
+    @Override
+    public void reset() throws Exception {
+        btree.create();
+    }
+
+    public class BTreeThread extends Thread {
+        private final DataGenThread dataGen;
+        private final int numBatches;
+        private final ITreeIndexAccessor indexAccessor;
+
+        public BTreeThread(DataGenThread dataGen, BTree btree, int numBatches) {
+            this.dataGen = dataGen;
+            this.numBatches = numBatches;
+            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        }
+
+        @Override
+        public void run() {
+            try {
+                for (int i = 0; i < numBatches; i++) {
+                    TupleBatch batch = dataGen.tupleBatchQueue.take();
+                    for (int j = 0; j < batch.size(); j++) {
+                        try {
+                            indexAccessor.insert(batch.get(j));
+                        } catch (TreeIndexException e) {
+                        }
+                    }
+                }
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java
new file mode 100644
index 0000000..53fbd88
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/InMemorySortRunner.java
@@ -0,0 +1,153 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.perf;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+
+public class InMemorySortRunner implements IExperimentRunner {
+    public class TupleComparator implements Comparator<ITupleReference> {
+        private final MultiComparator cmp;
+
+        public TupleComparator(MultiComparator cmp) {
+            this.cmp = cmp;
+        }
+
+        @Override
+        public int compare(ITupleReference o1, ITupleReference o2) {
+            return cmp.compare(o1, o2);
+        }
+    }
+    
+    private final TupleComparator tupleCmp;
+    private final int numBatches;
+    private final int batchSize;
+    private final int tupleSize;
+    private final ITypeTraits[] typeTraits;
+    
+    private final TypeAwareTupleWriterFactory tupleWriterFactory;
+    private final TypeAwareTupleWriter tupleWriter;
+    private final ArrayList<TypeAwareTupleReference> tuples;        
+    private final ByteBuffer tupleBuf; 
+    
+    public InMemorySortRunner(int numBatches, int batchSize, int tupleSize, ITypeTraits[] typeTraits, MultiComparator cmp) {
+        this.numBatches = numBatches;
+        this.tupleSize = tupleSize;
+        this.batchSize = batchSize;
+        this.typeTraits = typeTraits;
+        tupleCmp = new TupleComparator(cmp);
+        tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+        tupleWriter = (TypeAwareTupleWriter) tupleWriterFactory.createTupleWriter();
+        int numTuples = numBatches * batchSize;
+        tuples = new ArrayList<TypeAwareTupleReference>();
+        tupleBuf = ByteBuffer.allocate(numTuples * tupleSize);
+        for (int i = 0; i < numTuples; i++) {
+            tuples.add((TypeAwareTupleReference) tupleWriter.createTupleReference());
+        }
+    }
+    
+    @Override
+    public long runExperiment(DataGenThread dataGen, int numThreads) throws InterruptedException {
+        // Wait until the tupleBatchQueue is completely full.
+        while (dataGen.tupleBatchQueue.remainingCapacity() != 0) {
+            Thread.sleep(10);
+        }
+        
+        long start = System.currentTimeMillis();
+        int tupleIndex = 0;
+        for (int i = 0; i < numBatches; i++) {
+            TupleBatch batch = dataGen.tupleBatchQueue.take();
+            for (int j = 0; j < batch.size(); j++) {
+                // Copy the tuple to the buffer and set the pre-created tuple ref.                        
+                tupleWriter.writeTuple(batch.get(j), tupleBuf.array(), tupleIndex * tupleSize);
+                tuples.get(tupleIndex).resetByTupleOffset(tupleBuf, tupleIndex * tupleSize);
+                tupleIndex++;
+            }
+        }
+        // Perform the sort.        
+        Collections.sort(tuples, tupleCmp);
+        long end = System.currentTimeMillis();
+        long time = end - start;
+        return time;
+    }
+
+    @Override
+    public void init() throws Exception {
+    }
+
+    @Override
+    public void deinit() throws Exception {
+    }
+    
+    public void reset() throws Exception {
+    }
+    
+    public class SkipListThread extends Thread {
+    	private final DataGenThread dataGen;
+    	private final ConcurrentSkipListSet<ITupleReference> skipList;
+    	private final int numBatches;
+        public final TypeAwareTupleWriterFactory tupleWriterFactory;
+        public final TypeAwareTupleWriter tupleWriter;
+        public final TypeAwareTupleReference[] tuples;        
+        public final ByteBuffer tupleBuf; 
+
+        public SkipListThread(DataGenThread dataGen, ConcurrentSkipListSet<ITupleReference> skipList, int numBatches, int batchSize) {
+            this.dataGen = dataGen;
+            this.numBatches = numBatches;
+            this.skipList = skipList;
+            tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+            tupleWriter = (TypeAwareTupleWriter) tupleWriterFactory.createTupleWriter();
+            int numTuples = numBatches * batchSize;
+            tuples = new TypeAwareTupleReference[numTuples];
+            tupleBuf = ByteBuffer.allocate(numTuples * tupleSize);
+            for (int i = 0; i < numTuples; i++) {
+                tuples[i] = (TypeAwareTupleReference) tupleWriter.createTupleReference();
+            }
+        }
+    	
+        @Override
+        public void run() {
+            int tupleIndex = 0;
+            try {                
+                for (int i = 0; i < numBatches; i++) {
+                    TupleBatch batch = dataGen.tupleBatchQueue.take();
+                    for (int j = 0; j < batch.size(); j++) {
+                        // Copy the tuple to the buffer and set the pre-created tuple ref.                        
+                        tupleWriter.writeTuple(batch.get(j), tupleBuf.array(), tupleIndex * tupleSize);
+                        tuples[tupleIndex].resetByTupleOffset(tupleBuf, tupleIndex * tupleSize);
+                        skipList.add(tuples[tupleIndex]);
+                        tupleIndex++;
+                    }
+                }
+            } catch (Exception e) {
+                System.out.println(tupleIndex);
+                e.printStackTrace();
+            }
+        }
+    }
+}
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
new file mode 100644
index 0000000..5d2185a
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.perf;
+
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public class LSMTreeRunner implements IExperimentRunner {
+
+    private static final int MAX_OPEN_FILES = 10000;
+    private static final int HYRACKS_FRAME_SIZE = 128;
+
+    protected IHyracksTaskContext ctx;
+    protected IOManager ioManager;
+    protected IBufferCache bufferCache;
+    protected int lsmtreeFileId;
+
+    protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    protected final static String sep = System.getProperty("file.separator");
+    protected final static String classDir = "/lsmtree/";
+    protected String onDiskDir;
+    protected FileReference file;
+
+    protected final int numBatches;
+    protected final LSMBTree lsmtree;
+    protected final ILSMIOOperationScheduler ioScheduler;
+    protected IBufferCache memBufferCache;
+    private final int onDiskPageSize;
+    private final int onDiskNumPages;
+
+    public LSMTreeRunner(int numBatches, int inMemPageSize, int inMemNumPages, int onDiskPageSize, int onDiskNumPages,
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields)
+            throws BTreeException, HyracksException {
+        this.numBatches = numBatches;
+
+        this.onDiskPageSize = onDiskPageSize;
+        this.onDiskNumPages = onDiskNumPages;
+
+        onDiskDir = classDir + sep + simpleDateFormat.format(new Date()) + sep;
+        file = new FileReference(new File(onDiskDir));
+        ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+
+        TestStorageManagerComponentHolder.init(this.onDiskPageSize, this.onDiskNumPages, MAX_OPEN_FILES);
+        bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        ioManager = TestStorageManagerComponentHolder.getIOManager();
+        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+
+        IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), inMemPageSize,
+                inMemNumPages, new TransientFileMapManager());
+        IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(inMemNumPages,
+                new LIFOMetaDataFrameFactory());
+        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        lsmtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, bufferCache, fmp,
+                typeTraits, cmpFactories, bloomFilterKeyFields, NoMergePolicy.INSTANCE,
+                ThreadCountingOperationTrackerFactory.INSTANCE, ioScheduler, NoOpIOOperationCallback.INSTANCE);
+    }
+
+    @Override
+    public void init() throws Exception {
+    }
+
+    @Override
+    public long runExperiment(DataGenThread dataGen, int numThreads) throws Exception {
+        LSMTreeThread[] threads = new LSMTreeThread[numThreads];
+        int threadNumBatches = numBatches / numThreads;
+        for (int i = 0; i < numThreads; i++) {
+            threads[i] = new LSMTreeThread(dataGen, lsmtree, threadNumBatches);
+        }
+        // Wait until the tupleBatchQueue is completely full.
+        while (dataGen.tupleBatchQueue.remainingCapacity() != 0) {
+            Thread.sleep(10);
+        }
+
+        long start = System.currentTimeMillis();
+        for (int i = 0; i < numThreads; i++) {
+            threads[i].start();
+        }
+        for (int i = 0; i < numThreads; i++) {
+            threads[i].join();
+        }
+        long end = System.currentTimeMillis();
+        long time = end - start;
+        return time;
+    }
+
+    @Override
+    public void reset() throws Exception {
+        lsmtree.create();
+    }
+
+    @Override
+    public void deinit() throws Exception {
+        bufferCache.closeFile(lsmtreeFileId);
+        bufferCache.close();
+        memBufferCache.closeFile(lsmtreeFileId);
+        memBufferCache.close();
+    }
+
+    public class LSMTreeThread extends Thread {
+        private final DataGenThread dataGen;
+        private final int numBatches;
+        private final IIndexAccessor lsmTreeAccessor;
+
+        public LSMTreeThread(DataGenThread dataGen, LSMBTree lsmTree, int numBatches) {
+            this.dataGen = dataGen;
+            this.numBatches = numBatches;
+            lsmTreeAccessor = lsmTree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        }
+
+        @Override
+        public void run() {
+            try {
+                for (int i = 0; i < numBatches; i++) {
+                    TupleBatch batch = dataGen.tupleBatchQueue.take();
+                    for (int j = 0; j < batch.size(); j++) {
+                        try {
+                            lsmTreeAccessor.insert(batch.get(j));
+                        } catch (TreeIndexException e) {
+                        }
+                    }
+                }
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
new file mode 100644
index 0000000..c842191
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.perf;
+
+import java.util.Enumeration;
+import java.util.logging.Level;
+import java.util.logging.LogManager;
+import java.util.logging.Logger;
+
+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.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+
+public class PerfExperiment {
+    public static void main(String[] args) throws Exception {
+        // Disable logging so we can better see the output times.
+        Enumeration<String> loggers = LogManager.getLogManager().getLoggerNames();
+        while(loggers.hasMoreElements()) {
+            String loggerName = loggers.nextElement();
+            Logger logger = LogManager.getLogManager().getLogger(loggerName);
+            logger.setLevel(Level.OFF);
+        }
+        
+        int numTuples = 100000; // 100K
+        //int numTuples = 1000000; // 1M
+        //int numTuples = 2000000; // 2M
+        //int numTuples = 3000000; // 3M
+        //int numTuples = 10000000; // 10M
+        //int numTuples = 20000000; // 20M
+        //int numTuples = 30000000; // 30M
+        //int numTuples = 40000000; // 40M
+        //int numTuples = 60000000; // 60M
+        //int numTuples = 100000000; // 100M
+        //int numTuples = 200000000; // 200M
+        int batchSize = 10000;
+        int numBatches = numTuples / batchSize;
+        
+        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE };
+        ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes, 30);
+        
+        IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, fieldSerdes.length);
+        
+        //int repeats = 1000;
+        int repeats = 1;
+        long[] times = new long[repeats];
+
+        int numThreads = 2;
+        for (int i = 0; i < repeats; i++) {
+            //ConcurrentSkipListRunner runner = new ConcurrentSkipListRunner(numBatches, batchSize, tupleSize, typeTraits, cmp);
+            InMemoryBTreeRunner runner = new InMemoryBTreeRunner(numBatches, 8192, 100000, typeTraits, cmpFactories);
+            //BTreeBulkLoadRunner runner = new BTreeBulkLoadRunner(numBatches, 8192, 100000, typeTraits, cmp, 1.0f);
+        	//BTreeRunner runner = new BTreeRunner(numBatches, 8192, 100000, typeTraits, cmp);
+        	//String btreeName = "071211";
+        	//BTreeSearchRunner runner = new BTreeSearchRunner(btreeName, 10, numBatches, 8192, 25000, typeTraits, cmp);
+        	//LSMTreeRunner runner = new LSMTreeRunner(numBatches, 8192, 100, 8192, 250, typeTraits, cmp);
+        	//LSMTreeSearchRunner runner = new LSMTreeSearchRunner(100000, numBatches, 8192, 24750, 8192, 250, typeTraits, cmp); 
+            DataGenThread dataGen = new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, 30, 50, 10, false);
+            dataGen.start();
+            runner.reset();
+            times[i] = runner.runExperiment(dataGen, numThreads);
+            System.out.println("TIME " + i + ": " + times[i] + "ms");
+            runner.deinit();
+        }
+        long avgTime = 0;
+        for (int i = 0; i < repeats; i++) {
+            avgTime += times[i];
+        }
+        avgTime /= repeats;
+        System.out.println("AVG TIME: " + avgTime + "ms");
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
new file mode 100644
index 0000000..ce6c27c
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/tuples/LSMBTreeTuplesTest.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.tuples;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import org.junit.Test;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenUtils;
+import edu.uci.ics.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+
+@SuppressWarnings("rawtypes")
+public class LSMBTreeTuplesTest {
+
+    private final Random rnd = new Random(50);
+    
+    private ByteBuffer writeTuple(ITupleReference tuple, LSMBTreeTupleWriter tupleWriter) {
+        // Write tuple into a buffer, then later try to read it.
+        int bytesRequired = tupleWriter.bytesRequired(tuple);
+        byte[] bytes = new byte[bytesRequired];
+        ByteBuffer targetBuf = ByteBuffer.wrap(bytes);
+        tupleWriter.writeTuple(tuple, bytes, 0);
+        return targetBuf;
+    }
+    
+    private void testLSMBTreeTuple(ISerializerDeserializer[] maxFieldSerdes) throws HyracksDataException {        
+        // Create a tuple with the max-1 fields for checking setFieldCount() of tuple references later.
+        ITypeTraits[] maxTypeTraits = SerdeUtils.serdesToTypeTraits(maxFieldSerdes); 
+        IFieldValueGenerator[] maxFieldGens = DataGenUtils.getFieldGensFromSerdes(maxFieldSerdes, rnd, false);
+        // Generate a tuple with random field values.
+        Object[] maxFields = new Object[maxFieldSerdes.length];
+        for (int j = 0; j < maxFieldSerdes.length; j++) {
+            maxFields[j] = maxFieldGens[j].next();
+        }            
+        
+        // Run test for varying number of fields and keys.
+        for (int numKeyFields = 1; numKeyFields < maxFieldSerdes.length; numKeyFields++) {
+            // Create tuples with varying number of fields, and try to interpret their bytes with the lsmBTreeTuple.
+            for (int numFields = numKeyFields; numFields <= maxFieldSerdes.length; numFields++) {                
+                // Create and write tuple to bytes using an LSMBTreeTupleWriter.
+                LSMBTreeTupleWriter maxMatterTupleWriter = new LSMBTreeTupleWriter(maxTypeTraits, numKeyFields, false);
+                ITupleReference maxTuple = TupleUtils.createTuple(maxFieldSerdes, (Object[])maxFields);
+                ByteBuffer maxMatterBuf = writeTuple(maxTuple, maxMatterTupleWriter);
+                // Tuple reference should work for both matter and antimatter tuples (doesn't matter which factory creates it).
+                LSMBTreeTupleReference maxLsmBTreeTuple = (LSMBTreeTupleReference) maxMatterTupleWriter.createTupleReference();
+                
+                ISerializerDeserializer[] fieldSerdes = Arrays.copyOfRange(maxFieldSerdes, 0, numFields);
+                ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);                
+                IFieldValueGenerator[] fieldGens = DataGenUtils.getFieldGensFromSerdes(fieldSerdes, rnd, false);
+                // Generate a tuple with random field values.
+                Object[] fields = new Object[numFields];
+                for (int j = 0; j < numFields; j++) {
+                    fields[j] = fieldGens[j].next();
+                }            
+                // Create and write tuple to bytes using an LSMBTreeTupleWriter.
+                ITupleReference tuple = TupleUtils.createTuple(fieldSerdes, (Object[])fields);
+                LSMBTreeTupleWriter matterTupleWriter = new LSMBTreeTupleWriter(typeTraits, numKeyFields, false);
+                LSMBTreeTupleWriter antimatterTupleWriter = new LSMBTreeTupleWriter(typeTraits, numKeyFields, true);
+                LSMBTreeCopyTupleWriter copyTupleWriter = new LSMBTreeCopyTupleWriter(typeTraits, numKeyFields);
+                ByteBuffer matterBuf = writeTuple(tuple, matterTupleWriter);
+                ByteBuffer antimatterBuf = writeTuple(tuple, antimatterTupleWriter);
+
+                // The antimatter buf should only contain keys, sanity check the size.
+                if (numFields != numKeyFields) {
+                    assertTrue(antimatterBuf.array().length < matterBuf.array().length);
+                }
+
+                // Tuple reference should work for both matter and antimatter tuples (doesn't matter which factory creates it).
+                LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) matterTupleWriter.createTupleReference();                
+                
+                // Use LSMBTree tuple reference to interpret the written tuples.
+                // Repeat the block inside to test that repeated resetting to matter/antimatter tuples works.
+                for (int r = 0; r < 4; r++) {
+                    
+                    // Check matter tuple with lsmBTreeTuple.
+                    lsmBTreeTuple.resetByTupleOffset(matterBuf, 0);
+                    checkTuple(lsmBTreeTuple, numFields, false, fieldSerdes, fields);
+                    
+                    // Create a copy using copyTupleWriter, and verify again.
+                    ByteBuffer copyMatterBuf = writeTuple(lsmBTreeTuple, copyTupleWriter);
+                    lsmBTreeTuple.resetByTupleOffset(copyMatterBuf, 0);
+                    checkTuple(lsmBTreeTuple, numFields, false, fieldSerdes, fields);
+                    
+                    // Check antimatter tuple with lsmBTreeTuple.
+                    lsmBTreeTuple.resetByTupleOffset(antimatterBuf, 0);                                        
+                    // Should only contain keys.
+                    checkTuple(lsmBTreeTuple, numKeyFields, true, fieldSerdes, fields);
+                    
+                    // Create a copy using copyTupleWriter, and verify again.
+                    ByteBuffer copyAntimatterBuf = writeTuple(lsmBTreeTuple, copyTupleWriter);
+                    lsmBTreeTuple.resetByTupleOffset(copyAntimatterBuf, 0);
+                    // Should only contain keys.
+                    checkTuple(lsmBTreeTuple, numKeyFields, true, fieldSerdes, fields);
+                    
+                    // Check matter tuple with maxLsmBTreeTuple.
+                    // We should be able to manually set a prefix of the fields 
+                    // (the passed type traits in the tuple factory's constructor).
+                    maxLsmBTreeTuple.setFieldCount(numFields);
+                    maxLsmBTreeTuple.resetByTupleOffset(matterBuf, 0);
+                    checkTuple(maxLsmBTreeTuple, numFields, false, fieldSerdes, fields);
+                    
+                    // Check antimatter tuple with maxLsmBTreeTuple.
+                    maxLsmBTreeTuple.resetByTupleOffset(antimatterBuf, 0);
+                    // Should only contain keys.
+                    checkTuple(maxLsmBTreeTuple, numKeyFields, true, fieldSerdes, fields);
+                    
+                    // Resetting maxLsmBTreeTuple should set its field count to
+                    // maxFieldSerdes.length, based on the its type traits.
+                    maxLsmBTreeTuple.resetByTupleOffset(maxMatterBuf, 0);
+                    checkTuple(maxLsmBTreeTuple, maxFieldSerdes.length, false, maxFieldSerdes, maxFields);
+                }
+            }
+        }
+    }
+    
+    private void checkTuple(LSMBTreeTupleReference tuple, int expectedFieldCount, boolean expectedAntimatter, ISerializerDeserializer[] fieldSerdes, Object[] expectedFields) throws HyracksDataException {
+        assertEquals(expectedFieldCount, tuple.getFieldCount());
+        assertEquals(expectedAntimatter, tuple.isAntimatter());
+        Object[] deserMatterTuple = TupleUtils.deserializeTuple(tuple, fieldSerdes);
+        for (int j = 0; j < expectedFieldCount; j++) {
+            assertEquals(expectedFields[j], deserMatterTuple[j]);
+        }
+    }
+    
+    @Test
+    public void testLSMBTreeTuple() throws HyracksDataException {        
+        ISerializerDeserializer[] intFields = new IntegerSerializerDeserializer[] {
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE };
+        testLSMBTreeTuple(intFields);
+        
+        ISerializerDeserializer[] stringFields = new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        testLSMBTreeTuple(stringFields);
+        
+        ISerializerDeserializer[] mixedFields = new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE };
+        testLSMBTreeTuple(mixedFields);
+    }
+}
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
new file mode 100644
index 0000000..f790fde
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.util;
+
+import java.util.Collection;
+
+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.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;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+@SuppressWarnings("rawtypes")
+public final class LSMBTreeTestContext extends OrderedIndexTestContext {
+
+    public LSMBTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+        super(fieldSerdes, treeIndex);
+    }
+
+    @Override
+    public int getKeyFieldCount() {
+        LSMBTree lsmTree = (LSMBTree) index;
+        return lsmTree.getComparatorFactories().length;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getComparatorFactories() {
+        LSMBTree lsmTree = (LSMBTree) index;
+        return lsmTree.getComparatorFactories();
+    }
+
+    /**
+     * Override to provide upsert semantics for the check tuples.
+     */
+    @Override
+    public void insertCheckTuple(CheckTuple checkTuple, Collection<CheckTuple> checkTuples) {
+        upsertCheckTuple(checkTuple, checkTuples);
+    }
+
+    public static LSMBTreeTestContext create(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
+            int numKeyFields, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            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(memBufferCache, memFreePageManager, ioManager, file,
+                diskBufferCache, diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields, mergePolicy,
+                opTrackerFactory, 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
new file mode 100644
index 0000000..9128607
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -0,0 +1,215 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.btree.util;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Random;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public class LSMBTreeTestHarness {
+    protected static final Logger LOGGER = Logger.getLogger(LSMBTreeTestHarness.class.getName());
+
+    public static final BTreeLeafFrameType[] LEAF_FRAMES_TO_TEST = new BTreeLeafFrameType[] { BTreeLeafFrameType.REGULAR_NSM };
+
+    private static final long RANDOM_SEED = 50;
+
+    protected final int diskPageSize;
+    protected final int diskNumPages;
+    protected final int diskMaxOpenFiles;
+    protected final int memPageSize;
+    protected final int memNumPages;
+    protected final int hyracksFrameSize;
+
+    protected IOManager ioManager;
+    protected IBufferCache diskBufferCache;
+    protected IFileMapProvider diskFileMapProvider;
+    protected IInMemoryBufferCache memBufferCache;
+    protected IInMemoryFreePageManager memFreePageManager;
+    protected IHyracksTaskContext ctx;
+    protected ILSMIOOperationScheduler ioScheduler;
+    protected ILSMMergePolicy mergePolicy;
+    protected ILSMOperationTrackerFactory opTrackerFactory;
+    protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+
+    protected final Random rnd = new Random();
+    protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    protected final static String sep = System.getProperty("file.separator");
+    protected String onDiskDir;
+    protected FileReference file;
+
+    public LSMBTreeTestHarness() {
+        this.diskPageSize = AccessMethodTestsConfig.LSM_BTREE_DISK_PAGE_SIZE;
+        this.diskNumPages = AccessMethodTestsConfig.LSM_BTREE_DISK_NUM_PAGES;
+        this.diskMaxOpenFiles = AccessMethodTestsConfig.LSM_BTREE_DISK_MAX_OPEN_FILES;
+        this.memPageSize = AccessMethodTestsConfig.LSM_BTREE_MEM_PAGE_SIZE;
+        this.memNumPages = AccessMethodTestsConfig.LSM_BTREE_MEM_NUM_PAGES;
+        this.hyracksFrameSize = AccessMethodTestsConfig.LSM_BTREE_HYRACKS_FRAME_SIZE;
+        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.mergePolicy = NoMergePolicy.INSTANCE;
+        this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+        this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
+    }
+
+    public LSMBTreeTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
+            int memNumPages, int hyracksFrameSize) {
+        this.diskPageSize = diskPageSize;
+        this.diskNumPages = diskNumPages;
+        this.diskMaxOpenFiles = diskMaxOpenFiles;
+        this.memPageSize = memPageSize;
+        this.memNumPages = memNumPages;
+        this.hyracksFrameSize = hyracksFrameSize;
+        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.mergePolicy = NoMergePolicy.INSTANCE;
+        this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+    }
+
+    public void setUp() throws HyracksException {
+        onDiskDir = "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);
+        memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages,
+                new TransientFileMapManager());
+        memFreePageManager = new InMemoryFreePageManager(memNumPages, new LIFOMetaDataFrameFactory());
+        ioManager = TestStorageManagerComponentHolder.getIOManager();
+        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();
+                }
+            }
+            dir.delete();
+        }
+    }
+
+    public int getDiskPageSize() {
+        return diskPageSize;
+    }
+
+    public int getDiskNumPages() {
+        return diskNumPages;
+    }
+
+    public int getDiskMaxOpenFiles() {
+        return diskMaxOpenFiles;
+    }
+
+    public int getMemPageSize() {
+        return memPageSize;
+    }
+
+    public int getMemNumPages() {
+        return memNumPages;
+    }
+
+    public int getHyracksFrameSize() {
+        return hyracksFrameSize;
+    }
+
+    public IOManager getIOManager() {
+        return ioManager;
+    }
+
+    public IBufferCache getDiskBufferCache() {
+        return diskBufferCache;
+    }
+
+    public IFileMapProvider getDiskFileMapProvider() {
+        return diskFileMapProvider;
+    }
+
+    public IInMemoryBufferCache getMemBufferCache() {
+        return memBufferCache;
+    }
+
+    public IInMemoryFreePageManager getMemFreePageManager() {
+        return memFreePageManager;
+    }
+
+    public IHyracksTaskContext getHyracksTastContext() {
+        return ctx;
+    }
+
+    public FileReference getFileReference() {
+        return file;
+    }
+
+    public Random getRandom() {
+        return rnd;
+    }
+
+    public ILSMIOOperationScheduler getIOScheduler() {
+        return ioScheduler;
+    }
+
+    public ILSMOperationTrackerFactory getOperationTrackerFactory() {
+        return opTrackerFactory;
+    }
+
+    public ILSMMergePolicy getMergePolicy() {
+        return mergePolicy;
+    }
+
+    public ILSMIOOperationCallbackProvider getIOOperationCallbackProvider() {
+        return ioOpCallbackProvider;
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
new file mode 100644
index 0000000..27c16a6
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
@@ -0,0 +1,40 @@
+<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>
+  <artifactId>hyracks-storage-am-lsm-common-test</artifactId>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-tests</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>  	
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-lsm-common</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-test-support</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  </dependencies>
+</project>
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
new file mode 100644
index 0000000..69e23bc
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.util.ArrayList;
+
+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;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class DummyLSMIndexFileManager extends AbstractLSMIndexFileManager {
+
+    public DummyLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
+            TreeIndexFactory<? extends ITreeIndex> treeFactory) {
+        super(ioManager, fileMapProvider, file, treeFactory, 0);
+    }
+
+    protected void cleanupAndGetValidFilesInternal(IODeviceHandle dev, FilenameFilter filter,
+            TreeIndexFactory<? extends ITreeIndex> treeFactory, ArrayList<ComparableFileName> allFiles)
+            throws HyracksDataException, IndexException {
+        File dir = new File(dev.getPath(), baseDir);
+        String[] files = dir.list(filter);
+        for (String fileName : files) {
+            File file = new File(dir.getPath() + File.separator + fileName);
+            FileReference fileRef = new FileReference(file);
+            allFiles.add(new ComparableFileName(fileRef));
+        }
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyTreeFactory.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyTreeFactory.java
new file mode 100644
index 0000000..8b22771
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyTreeFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common;
+
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.TreeIndexFactory;
+
+public class DummyTreeFactory extends TreeIndexFactory<ITreeIndex> {
+
+    public DummyTreeFactory() {
+        super(null, null, null, null, null, null, 0);
+    }
+
+    @Override
+    public ITreeIndex createIndexInstance(FileReference file) throws IndexException {
+        return null;
+    }
+
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryBufferCacheTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryBufferCacheTest.java
new file mode 100644
index 0000000..adba93d
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryBufferCacheTest.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.HashSet;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+
+public class InMemoryBufferCacheTest {
+    private static final int PAGE_SIZE = 256;
+    private static final int NUM_PAGES = 100;
+    private HashSet<ICachedPage> pinnedPages = new HashSet<ICachedPage>();
+
+    @Test
+    public void test01() throws Exception {
+        InMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), PAGE_SIZE, NUM_PAGES,
+                new TransientFileMapManager());
+        memBufferCache.open();
+        int dummyFileId = 0;
+        // Pin all pages, and make sure they return unique ICachedPages.
+        // We expect no overflow pages.
+        for (int i = 0; i < NUM_PAGES; i++) {
+            ICachedPage page = memBufferCache.pin(BufferedFileHandle.getDiskPageId(dummyFileId, i), false);
+            if (pinnedPages.contains(page)) {
+                fail("Id collision for ICachedPage, caused by id: " + i);
+            }
+            pinnedPages.add(page);
+            assertEquals(0, memBufferCache.getNumOverflowPages());
+        }
+        // Pin pages above capacity. We expect to be given new overflow pages.
+        // Going above capacity should be very rare, but nevertheless succeed.
+        for (int i = 0; i < 100; i++) {
+            ICachedPage page = memBufferCache.pin(BufferedFileHandle.getDiskPageId(dummyFileId, i + NUM_PAGES), false);
+            if (pinnedPages.contains(page)) {
+                fail("Id collision for ICachedPage, caused by overflow id: " + i);
+            }
+            pinnedPages.add(page);
+            assertEquals(i + 1, memBufferCache.getNumOverflowPages());
+        }
+        memBufferCache.close();
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryFreePageManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryFreePageManagerTest.java
new file mode 100644
index 0000000..bd09a3f
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/InMemoryFreePageManagerTest.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+
+public class InMemoryFreePageManagerTest {
+
+    private final int NUM_PAGES = 100;
+    
+    private void testInMemoryFreePageManager(InMemoryFreePageManager memFreePageManager) throws HyracksDataException {
+        // The first two pages are reserved for the BTree's metadata page and
+        // root page.
+        // The "actual" capacity is therefore numPages - 2.
+        int capacity = memFreePageManager.getCapacity();
+        assertEquals(capacity, NUM_PAGES - 2);
+        for (int i = 0; i < capacity; i++) {
+            int pageId = memFreePageManager.getFreePage(null);
+            // The free pages start from page 2;
+            assertEquals(i + 2, pageId);
+            assertFalse(memFreePageManager.isFull());
+        }
+        // Start asking for 100 pages above the capacity.
+        // Asking for pages above the capacity should be very rare, but
+        // nevertheless succeed.
+        // We expect isFull() to return true.
+        for (int i = 0; i < 100; i++) {
+            int pageId = memFreePageManager.getFreePage(null);
+            assertEquals(capacity + i + 2, pageId);
+            assertTrue(memFreePageManager.isFull());
+        }
+    }
+    
+    @Test
+    public void test01() throws HyracksDataException {
+        ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+        InMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(NUM_PAGES, metaFrameFactory);
+        testInMemoryFreePageManager(memFreePageManager);
+        // We expect exactly the same behavior after a reset().
+        memFreePageManager.reset();
+        testInMemoryFreePageManager(memFreePageManager);
+    }
+}
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
new file mode 100644
index 0000000..161f4ce
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
@@ -0,0 +1,274 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.lsm.common;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.Executors;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+
+public class LSMIndexFileManagerTest {
+    private static final int DEFAULT_PAGE_SIZE = 256;
+    private static final int DEFAULT_NUM_PAGES = 100;
+    private static final int DEFAULT_MAX_OPEN_FILES = 10;
+    protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    protected final static String sep = System.getProperty("file.separator");
+    protected IOManager ioManager;
+    protected IFileMapProvider fileMapProvider;
+    protected String baseDir;
+    protected FileReference file;
+
+    @Before
+    public void setUp() throws HyracksException {
+        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;
+        File f = new File(baseDir);
+        f.mkdirs();
+        file = new FileReference(f);
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        File f = new File(baseDir);
+        f.deleteOnExit();
+    }
+
+    public void sortOrderTest(boolean testFlushFileName) throws InterruptedException, HyracksDataException {
+        ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
+                new DummyTreeFactory());
+        LinkedList<String> fileNames = new LinkedList<String>();
+
+        int numFileNames = 100;
+        long sleepTime = 5;
+        for (int i = 0; i < numFileNames; i++) {
+            String flushFileName = (String) fileManager.getRelFlushFileReference().getInsertIndexFileReference()
+                    .getFile().getName();
+            if (testFlushFileName) {
+                fileNames.addFirst(flushFileName);
+            }
+            Thread.sleep(sleepTime);
+            if (!testFlushFileName) {
+                String secondFlushFileName = (String) fileManager.getRelFlushFileReference()
+                        .getInsertIndexFileReference().getFile().getName();
+                String mergeFileName = getMergeFileName(fileManager, flushFileName, secondFlushFileName);
+                fileNames.addFirst(mergeFileName);
+                Thread.sleep(sleepTime);
+            }
+        }
+
+        List<String> sortedFileNames = new ArrayList<String>();
+        sortedFileNames.addAll(fileNames);
+
+        // Make sure the comparator sorts in the correct order (i.e., the
+        // reverse insertion order in this case).
+        Comparator<String> cmp = fileManager.getFileNameComparator();
+        Collections.sort(sortedFileNames, cmp);
+        for (int i = 0; i < numFileNames; i++) {
+            assertEquals(fileNames.get(i), sortedFileNames.get(i));
+        }
+    }
+
+    @Test
+    public void flushAndMergeFilesSortOrderTest() throws InterruptedException, HyracksDataException {
+        sortOrderTest(true);
+        sortOrderTest(false);
+    }
+
+    public void cleanInvalidFilesTest(IOManager ioManager) throws InterruptedException, IOException, IndexException {
+        ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
+                new DummyTreeFactory());
+        fileManager.createDirs();
+
+        List<FileReference> flushFiles = new ArrayList<FileReference>();
+        List<FileReference> allFiles = new ArrayList<FileReference>();
+
+        int numFileNames = 100;
+        long sleepTime = 5;
+        // Generate a bunch of flush files.
+        for (int i = 0; i < numFileNames; i++) {
+            LSMComponentFileReferences relFlushFileRefs = fileManager.getRelFlushFileReference();
+            flushFiles.add(relFlushFileRefs.getInsertIndexFileReference());
+            Thread.sleep(sleepTime);
+        }
+        allFiles.addAll(flushFiles);
+
+        // Simulate merging some of the flush files.
+        // Merge range 0 to 4.
+        FileReference mergeFile1 = simulateMerge(fileManager, flushFiles.get(0), flushFiles.get(4));
+        allFiles.add(mergeFile1);
+        // Merge range 5 to 9.
+        FileReference mergeFile2 = simulateMerge(fileManager, flushFiles.get(5), flushFiles.get(9));
+        allFiles.add(mergeFile2);
+        // Merge range 10 to 19.
+        FileReference mergeFile3 = simulateMerge(fileManager, flushFiles.get(10), flushFiles.get(19));
+        allFiles.add(mergeFile3);
+        // Merge range 20 to 29.
+        FileReference mergeFile4 = simulateMerge(fileManager, flushFiles.get(20), flushFiles.get(29));
+        allFiles.add(mergeFile4);
+        // Merge range 50 to 79.
+        FileReference mergeFile5 = simulateMerge(fileManager, flushFiles.get(50), flushFiles.get(79));
+        allFiles.add(mergeFile5);
+
+        // Simulate merging of merge files.
+        FileReference mergeFile6 = simulateMerge(fileManager, mergeFile1, mergeFile2);
+        allFiles.add(mergeFile6);
+        FileReference mergeFile7 = simulateMerge(fileManager, mergeFile3, mergeFile4);
+        allFiles.add(mergeFile7);
+
+        // Create all files and set delete on exit for all files.
+        for (FileReference fileRef : allFiles) {
+            fileRef.getFile().createNewFile();
+            fileRef.getFile().deleteOnExit();
+        }
+
+        // Populate expected valid flush files.
+        List<String> expectedValidFiles = new ArrayList<String>();
+        for (int i = 30; i < 50; i++) {
+            expectedValidFiles.add(flushFiles.get(i).getFile().getName());
+        }
+        for (int i = 80; i < 100; i++) {
+            expectedValidFiles.add(flushFiles.get(i).getFile().getName());
+        }
+
+        // Populate expected valid merge files.
+        expectedValidFiles.add(mergeFile5.getFile().getName());
+        expectedValidFiles.add(mergeFile6.getFile().getName());
+        expectedValidFiles.add(mergeFile7.getFile().getName());
+
+        // Sort expected files.
+        Collections.sort(expectedValidFiles, fileManager.getFileNameComparator());
+
+        // Pass null and a dummy component finalizer. We don't test for physical consistency in this test.
+        List<LSMComponentFileReferences> lsmComonentFileReference = fileManager.cleanupAndGetValidFiles();
+
+        // Check actual files against expected files.
+        assertEquals(expectedValidFiles.size(), lsmComonentFileReference.size());
+        for (int i = 0; i < expectedValidFiles.size(); i++) {
+            assertEquals(expectedValidFiles.get(i), lsmComonentFileReference.get(i).getInsertIndexFileReference()
+                    .getFile().getName());
+        }
+
+        // Make sure invalid files were removed from all IODevices.
+        ArrayList<String> remainingFiles = new ArrayList<String>();
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            File dir = new File(dev.getPath(), baseDir);
+            FilenameFilter filter = new FilenameFilter() {
+                public boolean accept(File dir, String name) {
+                    return !name.startsWith(".");
+                }
+            };
+            String[] files = dir.list(filter);
+            for (String file : files) {
+                File f = new File(file);
+                remainingFiles.add(f.getName());
+            }
+        }
+
+        Collections.sort(remainingFiles, fileManager.getFileNameComparator());
+        // Check actual files in directory against expected files.
+        assertEquals(expectedValidFiles.size(), remainingFiles.size());
+        for (int i = 0; i < expectedValidFiles.size(); i++) {
+            assertEquals(expectedValidFiles.get(i), remainingFiles.get(i));
+        }
+    }
+
+    @Test
+    public void singleIODeviceTest() throws InterruptedException, IOException, IndexException {
+        IOManager singleDeviceIOManager = createIOManager(1);
+        cleanInvalidFilesTest(singleDeviceIOManager);
+        cleanDirs(singleDeviceIOManager);
+    }
+
+    @Test
+    public void twoIODevicesTest() throws InterruptedException, IOException, IndexException {
+        IOManager twoDevicesIOManager = createIOManager(2);
+        cleanInvalidFilesTest(twoDevicesIOManager);
+        cleanDirs(twoDevicesIOManager);
+    }
+
+    @Test
+    public void fourIODevicesTest() throws InterruptedException, IOException, IndexException {
+        IOManager fourDevicesIOManager = createIOManager(4);
+        cleanInvalidFilesTest(fourDevicesIOManager);
+        cleanDirs(fourDevicesIOManager);
+    }
+
+    private void cleanDirs(IOManager ioManager) {
+        for (IODeviceHandle dev : ioManager.getIODevices()) {
+            File dir = new File(dev.getPath(), baseDir);
+            FilenameFilter filter = new FilenameFilter() {
+                public boolean accept(File dir, String name) {
+                    return !name.startsWith(".");
+                }
+            };
+            String[] files = dir.list(filter);
+            for (String file : files) {
+                File f = new File(file);
+                f.delete();
+            }
+        }
+    }
+
+    private IOManager createIOManager(int numDevices) throws HyracksException {
+        List<IODeviceHandle> devices = new ArrayList<IODeviceHandle>();
+        for (int i = 0; i < numDevices; i++) {
+            String iodevPath = System.getProperty("java.io.tmpdir") + sep + "test_iodev" + i;
+            devices.add(new IODeviceHandle(new File(iodevPath), "wa"));
+        }
+        return new IOManager(devices, Executors.newCachedThreadPool());
+    }
+
+    private FileReference simulateMerge(ILSMIndexFileManager fileManager, FileReference a, FileReference b)
+            throws HyracksDataException {
+        LSMComponentFileReferences relMergeFileRefs = fileManager.getRelMergeFileReference(a.getFile().getName(), b
+                .getFile().getName());
+        return relMergeFileRefs.getInsertIndexFileReference();
+    }
+
+    private String getMergeFileName(ILSMIndexFileManager fileNameManager, String firstFile, String lastFile)
+            throws HyracksDataException {
+        File f1 = new File(firstFile);
+        File f2 = new File(lastFile);
+        return (String) fileNameManager.getRelMergeFileReference(f1.getName(), f2.getName())
+                .getInsertIndexFileReference().getFile().getName();
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
new file mode 100644
index 0000000..d2932d6
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
@@ -0,0 +1,49 @@
+<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>hyracks-storage-am-lsm-invertedindex-test</artifactId>
+
+	<parent>
+		<artifactId>hyracks-tests</artifactId>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<version>0.2.3-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>2.0.2</version>
+				<configuration>
+					<source>1.7</source>
+					<target>1.7</target>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+	<dependencies>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-test-support</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-data-std</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+</project>
\ No newline at end of file
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
new file mode 100644
index 0000000..fcb78ad
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexBulkLoadTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class LSMInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
+
+    public LSMInvertedIndexBulkLoadTest() {
+        super(InvertedIndexType.LSM, true, 1);
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexDeleteTest.java
new file mode 100644
index 0000000..4e2fe37
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexDeleteTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.lsm.invertedindex.common.AbstractInvertedIndexDeleteTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class LSMInvertedIndexDeleteTest extends AbstractInvertedIndexDeleteTest {
+
+    public LSMInvertedIndexDeleteTest() {
+        super(InvertedIndexType.LSM, false);
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..e9a1c75
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexInsertTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class LSMInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
+
+    public LSMInvertedIndexInsertTest() {
+        super(InvertedIndexType.LSM, false, 1);
+    }
+}
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
new file mode 100644
index 0000000..811919b
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2012 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 java.io.IOException;
+
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleGenerator;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils;
+
+public class LSMInvertedIndexMergeTest extends AbstractInvertedIndexLoadTest {
+
+    private final int maxTreesToMerge = AccessMethodTestsConfig.LSM_INVINDEX_MAX_TREES_TO_MERGE;
+
+    public LSMInvertedIndexMergeTest() {
+        super(InvertedIndexType.LSM, true, 1);
+    }
+
+    @Override
+    protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen) throws IOException,
+            IndexException {
+        IIndex invIndex = testCtx.getIndex();
+        invIndex.create();
+        invIndex.activate();
+        ILSMIndexAccessor invIndexAccessor = (ILSMIndexAccessor) invIndex.createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+
+        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);
+                }
+            }
+            // Perform merge.
+            invIndexAccessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+            validateAndCheckIndex(testCtx);
+            runTinySearchWorkload(testCtx, tupleGen);
+        }
+
+        invIndex.deactivate();
+        invIndex.destroy();
+    }
+}
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
new file mode 100644
index 0000000..adfb689
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMultiBulkLoadTest.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2012 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/LSMInvertedIndexSearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexSearchTest.java
new file mode 100644
index 0000000..1528e20
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexSearchTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.lsm.invertedindex.common.AbstractInvertedIndexSearchTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class LSMInvertedIndexSearchTest extends AbstractInvertedIndexSearchTest {
+
+    public LSMInvertedIndexSearchTest() {
+        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/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
new file mode 100644
index 0000000..f7a36f0
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
+
+    public PartitionedLSMInvertedIndexBulkLoadTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, true, 1);
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexDeleteTest.java
new file mode 100644
index 0000000..4fd529b
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexDeleteTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.lsm.invertedindex.common.AbstractInvertedIndexDeleteTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexDeleteTest extends AbstractInvertedIndexDeleteTest {
+
+    public PartitionedLSMInvertedIndexDeleteTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, false);
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..4608f81
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
+
+    public PartitionedLSMInvertedIndexInsertTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, false, 1);
+    }
+}
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
new file mode 100644
index 0000000..786afe1
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2012 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 java.io.IOException;
+
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleGenerator;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils;
+
+public class PartitionedLSMInvertedIndexMergeTest extends AbstractInvertedIndexLoadTest {
+
+    private final int maxTreesToMerge = AccessMethodTestsConfig.LSM_INVINDEX_MAX_TREES_TO_MERGE;
+
+    public PartitionedLSMInvertedIndexMergeTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, true, 1);
+    }
+
+    @Override
+    protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen) throws IOException,
+            IndexException {
+        IIndex invIndex = testCtx.getIndex();
+        invIndex.create();
+        invIndex.activate();
+        ILSMIndexAccessor invIndexAccessor = (ILSMIndexAccessor) invIndex.createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+
+        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);
+                }
+            }
+            // Perform merge.
+            invIndexAccessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+            validateAndCheckIndex(testCtx);
+            runTinySearchWorkload(testCtx, tupleGen);
+        }
+
+        invIndex.deactivate();
+        invIndex.destroy();
+    }
+}
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
new file mode 100644
index 0000000..80a3c0b
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2012 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/PartitionedLSMInvertedIndexSearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexSearchTest.java
new file mode 100644
index 0000000..c8a7667
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexSearchTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.lsm.invertedindex.common.AbstractInvertedIndexSearchTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexSearchTest extends AbstractInvertedIndexSearchTest {
+
+    public PartitionedLSMInvertedIndexSearchTest() {
+        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/common/AbstractInvertedIndexDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexDeleteTest.java
new file mode 100644
index 0000000..f7783fb
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexDeleteTest.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2009-2012 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.common;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleGenerator;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils;
+
+public abstract class AbstractInvertedIndexDeleteTest extends AbstractInvertedIndexTest {
+
+    protected final int numInsertRounds = AccessMethodTestsConfig.LSM_INVINDEX_NUM_INSERT_ROUNDS;
+    protected final int numDeleteRounds = AccessMethodTestsConfig.LSM_INVINDEX_NUM_DELETE_ROUNDS;
+    protected final boolean bulkLoad;
+
+    public AbstractInvertedIndexDeleteTest(InvertedIndexType invIndexType, boolean bulkLoad) {
+        super(invIndexType);
+        this.bulkLoad = bulkLoad;
+    }
+
+    protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen) throws IOException,
+            IndexException {
+        IIndex invIndex = testCtx.getIndex();
+        invIndex.create();
+        invIndex.activate();
+
+        for (int i = 0; i < numInsertRounds; i++) {
+            // Start generating documents ids from 0 again.
+            tupleGen.reset();
+
+            if (bulkLoad) {
+                LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+            } else {
+                LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+            }
+
+            // Delete all documents in a couple of rounds.
+            int numTuplesPerDeleteRound = (int) Math.ceil((float) testCtx.getDocumentCorpus().size()
+                    / (float) numDeleteRounds);
+            for (int j = 0; j < numDeleteRounds; j++) {
+                LSMInvertedIndexTestUtils.deleteFromInvIndex(testCtx, harness.getRandom(), numTuplesPerDeleteRound);
+                validateAndCheckIndex(testCtx);
+                runTinySearchWorkload(testCtx, tupleGen);
+            }
+        }
+
+        invIndex.deactivate();
+        invIndex.destroy();
+    }
+
+    @Test
+    public void wordTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createWordInvIndexTestContext(harness, invIndexType);
+        TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createStringDocumentTupleGen(harness.getRandom());
+        runTest(testCtx, tupleGen);
+    }
+
+    @Test
+    public void hashedWordTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createHashedWordInvIndexTestContext(harness,
+                invIndexType);
+        TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createStringDocumentTupleGen(harness.getRandom());
+        runTest(testCtx, tupleGen);
+    }
+
+    @Test
+    public void ngramTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createNGramInvIndexTestContext(harness, invIndexType);
+        TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createPersonNamesTupleGen(harness.getRandom());
+        runTest(testCtx, tupleGen);
+    }
+
+    @Test
+    public void hashedNGramTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createHashedNGramInvIndexTestContext(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/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
new file mode 100644
index 0000000..c855cc4
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexLoadTest.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2012 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.common;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleGenerator;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils;
+
+public abstract class AbstractInvertedIndexLoadTest extends AbstractInvertedIndexTest {
+
+    protected final boolean bulkLoad;
+    protected final int numRounds;
+
+    public AbstractInvertedIndexLoadTest(InvertedIndexType invIndexType, boolean bulkLoad, int numRounds) {
+        super(invIndexType);
+        this.bulkLoad = bulkLoad;
+        this.numRounds = numRounds;
+    }
+
+    protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen) throws IOException,
+            IndexException {
+        IIndex invIndex = testCtx.getIndex();
+        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);
+        }
+
+        invIndex.deactivate();
+        invIndex.destroy();
+    }
+
+    @Test
+    public void wordTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createWordInvIndexTestContext(harness, invIndexType);
+        TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createStringDocumentTupleGen(harness.getRandom());
+        runTest(testCtx, tupleGen);
+    }
+
+    @Test
+    public void hashedWordTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createHashedWordInvIndexTestContext(harness,
+                invIndexType);
+        TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createStringDocumentTupleGen(harness.getRandom());
+        runTest(testCtx, tupleGen);
+    }
+
+    @Test
+    public void ngramTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createNGramInvIndexTestContext(harness, invIndexType);
+        TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createPersonNamesTupleGen(harness.getRandom());
+        runTest(testCtx, tupleGen);
+    }
+
+    @Test
+    public void hashedNGramTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createHashedNGramInvIndexTestContext(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/AbstractInvertedIndexSearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexSearchTest.java
new file mode 100644
index 0000000..991ff59
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexSearchTest.java
@@ -0,0 +1,128 @@
+/*
+ * Copyright 2009-2012 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.common;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleGenerator;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.EditDistanceSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils;
+
+public abstract class AbstractInvertedIndexSearchTest extends AbstractInvertedIndexTest {
+
+    protected final Logger LOGGER = Logger.getLogger(AbstractInvertedIndexSearchTest.class.getName());
+
+    protected int NUM_DOC_QUERIES = AccessMethodTestsConfig.LSM_INVINDEX_NUM_DOC_QUERIES;
+    protected int NUM_RANDOM_QUERIES = AccessMethodTestsConfig.LSM_INVINDEX_NUM_RANDOM_QUERIES;
+    protected final boolean bulkLoad;
+
+    public AbstractInvertedIndexSearchTest(InvertedIndexType invIndexType, boolean bulkLoad) {
+        super(invIndexType);
+        this.bulkLoad = bulkLoad;
+    }
+
+    protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen,
+            List<IInvertedIndexSearchModifier> searchModifiers) throws IOException, IndexException {
+        IIndex invIndex = testCtx.getIndex();
+        invIndex.create();
+        invIndex.activate();
+
+        if (bulkLoad) {
+            LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+        } else {
+            LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+        }
+        invIndex.validate();
+
+        for (IInvertedIndexSearchModifier searchModifier : searchModifiers) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Running searches with: " + searchModifier.toString());
+            }
+            LSMInvertedIndexTestUtils.testIndexSearch(testCtx, tupleGen, harness.getRandom(), NUM_DOC_QUERIES,
+                    NUM_RANDOM_QUERIES, searchModifier, SCAN_COUNT_ARRAY);
+        }
+
+        invIndex.deactivate();
+        invIndex.destroy();
+    }
+
+    private void testWordInvIndexIndex(LSMInvertedIndexTestContext testCtx) throws IOException, IndexException {
+        TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createStringDocumentTupleGen(harness.getRandom());
+        List<IInvertedIndexSearchModifier> searchModifiers = new ArrayList<IInvertedIndexSearchModifier>();
+        searchModifiers.add(new ConjunctiveSearchModifier());
+        searchModifiers.add(new JaccardSearchModifier(1.0f));
+        searchModifiers.add(new JaccardSearchModifier(0.9f));
+        searchModifiers.add(new JaccardSearchModifier(0.7f));
+        searchModifiers.add(new JaccardSearchModifier(0.5f));
+        runTest(testCtx, tupleGen, searchModifiers);
+    }
+
+    private void testNGramInvIndexIndex(LSMInvertedIndexTestContext testCtx) throws IOException, IndexException {
+        TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createPersonNamesTupleGen(harness.getRandom());
+        List<IInvertedIndexSearchModifier> searchModifiers = new ArrayList<IInvertedIndexSearchModifier>();
+        searchModifiers.add(new ConjunctiveSearchModifier());
+        searchModifiers.add(new JaccardSearchModifier(1.0f));
+        searchModifiers.add(new JaccardSearchModifier(0.9f));
+        searchModifiers.add(new JaccardSearchModifier(0.7f));
+        searchModifiers.add(new JaccardSearchModifier(0.5f));
+        searchModifiers.add(new EditDistanceSearchModifier(LSMInvertedIndexTestUtils.TEST_GRAM_LENGTH, 0));
+        searchModifiers.add(new EditDistanceSearchModifier(LSMInvertedIndexTestUtils.TEST_GRAM_LENGTH, 1));
+        searchModifiers.add(new EditDistanceSearchModifier(LSMInvertedIndexTestUtils.TEST_GRAM_LENGTH, 2));
+        searchModifiers.add(new EditDistanceSearchModifier(LSMInvertedIndexTestUtils.TEST_GRAM_LENGTH, 3));
+        runTest(testCtx, tupleGen, searchModifiers);
+    }
+
+    @Test
+    public void wordTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createWordInvIndexTestContext(harness, invIndexType);
+        testWordInvIndexIndex(testCtx);
+    }
+
+    @Test
+    public void hashedWordTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createHashedWordInvIndexTestContext(harness,
+                invIndexType);
+        testWordInvIndexIndex(testCtx);
+    }
+
+    @Test
+    public void ngramTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createNGramInvIndexTestContext(harness, invIndexType);
+        testNGramInvIndexIndex(testCtx);
+    }
+
+    @Test
+    public void hashedNGramTokensInvIndexTest() throws IOException, IndexException {
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createHashedNGramInvIndexTestContext(harness,
+                invIndexType);
+        testNGramInvIndexIndex(testCtx);
+    }
+
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
new file mode 100644
index 0000000..90a6d54
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
@@ -0,0 +1,101 @@
+/*
+ * Copyright 2009-2012 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.common;
+
+import java.io.IOException;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.junit.After;
+import org.junit.Before;
+
+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.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleGenerator;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils;
+
+public abstract class AbstractInvertedIndexTest {
+    protected final Logger LOGGER = Logger.getLogger(AbstractInvertedIndexTest.class.getName());
+
+    protected final LSMInvertedIndexTestHarness harness = new LSMInvertedIndexTestHarness();
+
+    protected final int NUM_DOCS_TO_INSERT = AccessMethodTestsConfig.LSM_INVINDEX_NUM_DOCS_TO_INSERT;
+    protected final int[] SCAN_COUNT_ARRAY = new int[AccessMethodTestsConfig.LSM_INVINDEX_SCAN_COUNT_ARRAY_SIZE];
+
+    protected final int TINY_WORKLOAD_NUM_DOC_QUERIES = AccessMethodTestsConfig.LSM_INVINDEX_TINY_NUM_DOC_QUERIES;
+    protected final int TINY_WORKLOAD_NUM_RANDOM_QUERIES = AccessMethodTestsConfig.LSM_INVINDEX_TINY_NUM_RANDOM_QUERIES;
+
+    // Note: The edit-distance search modifier is tested separately.
+    protected final IInvertedIndexSearchModifier[] TEST_SEARCH_MODIFIERS = new IInvertedIndexSearchModifier[] {
+            new ConjunctiveSearchModifier(), new JaccardSearchModifier(0.8f), new JaccardSearchModifier(0.5f) };
+
+    protected final InvertedIndexType invIndexType;
+
+    public AbstractInvertedIndexTest(InvertedIndexType invIndexType) {
+        this.invIndexType = invIndexType;
+    }
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    /**
+     * Validates the index, and compares it against the expected index.
+     * This test is only for verifying the integrity and correctness of the index,
+     * it does not ensure the correctness of index searches.
+     */
+    protected void validateAndCheckIndex(LSMInvertedIndexTestContext testCtx) throws HyracksDataException, IndexException {
+        IIndex invIndex = testCtx.getIndex();
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Validating index: " + invIndex);
+        }
+        // Validate index and compare against expected index.
+        invIndex.validate();
+        if (invIndexType == InvertedIndexType.INMEMORY || invIndexType == InvertedIndexType.ONDISK) {
+            // This comparison method exercises different features of these types of inverted indexes.
+            LSMInvertedIndexTestUtils.compareActualAndExpectedIndexes(testCtx);
+        }
+        LSMInvertedIndexTestUtils.compareActualAndExpectedIndexesRangeSearch(testCtx);
+    }
+
+    /**
+     * Runs a workload of queries using different search modifiers, and verifies the correctness of the results.
+     */
+    protected void runTinySearchWorkload(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen) throws IOException,
+            IndexException {
+        for (IInvertedIndexSearchModifier searchModifier : TEST_SEARCH_MODIFIERS) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Running test workload with: " + searchModifier.toString());
+            }
+            LSMInvertedIndexTestUtils.testIndexSearch(testCtx, tupleGen, harness.getRandom(),
+                    TINY_WORKLOAD_NUM_DOC_QUERIES, TINY_WORKLOAD_NUM_RANDOM_QUERIES, searchModifier, SCAN_COUNT_ARRAY);
+        }
+    }
+}
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
new file mode 100644
index 0000000..5be1d6a
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2009-2010 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.common;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Random;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public class LSMInvertedIndexTestHarness {
+
+    private static final long RANDOM_SEED = 50;
+
+    protected final int diskPageSize;
+    protected final int diskNumPages;
+    protected final int diskMaxOpenFiles;
+    protected final int memPageSize;
+    protected final int memNumPages;
+    protected final int hyracksFrameSize;
+
+    protected IOManager ioManager;
+    protected IBufferCache diskBufferCache;
+    protected IFileMapProvider diskFileMapProvider;
+    protected IInMemoryBufferCache memBufferCache;
+    protected IInMemoryFreePageManager memFreePageManager;
+    protected IHyracksTaskContext ctx;
+    protected ILSMIOOperationScheduler ioScheduler;
+    protected ILSMMergePolicy mergePolicy;
+    protected ILSMOperationTrackerFactory opTrackerFactory;
+    protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+
+    protected final Random rnd = new Random();
+    protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    protected final static String sep = System.getProperty("file.separator");
+    protected String onDiskDir;
+    protected String btreeFileName = "btree_vocab";
+    protected String invIndexFileName = "inv_index";
+    protected FileReference invIndexFileRef;
+
+    public LSMInvertedIndexTestHarness() {
+        this.diskPageSize = AccessMethodTestsConfig.LSM_INVINDEX_DISK_PAGE_SIZE;
+        this.diskNumPages = AccessMethodTestsConfig.LSM_INVINDEX_DISK_NUM_PAGES;
+        this.diskMaxOpenFiles = AccessMethodTestsConfig.LSM_INVINDEX_DISK_MAX_OPEN_FILES;
+        this.memPageSize = AccessMethodTestsConfig.LSM_INVINDEX_MEM_PAGE_SIZE;
+        this.memNumPages = AccessMethodTestsConfig.LSM_INVINDEX_MEM_NUM_PAGES;
+        this.hyracksFrameSize = AccessMethodTestsConfig.LSM_INVINDEX_HYRACKS_FRAME_SIZE;
+        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.mergePolicy = NoMergePolicy.INSTANCE;
+        this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+        this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
+    }
+
+    public LSMInvertedIndexTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
+            int memNumPages, int hyracksFrameSize) {
+        this.diskPageSize = diskPageSize;
+        this.diskNumPages = diskNumPages;
+        this.diskMaxOpenFiles = diskMaxOpenFiles;
+        this.memPageSize = memPageSize;
+        this.memNumPages = memNumPages;
+        this.hyracksFrameSize = hyracksFrameSize;
+        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.mergePolicy = NoMergePolicy.INSTANCE;
+        this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+    }
+
+    public void setUp() throws HyracksException {
+        onDiskDir = "lsm_invertedindex_" + simpleDateFormat.format(new Date()) + sep;
+        ctx = TestUtils.create(getHyracksFrameSize());
+        TestStorageManagerComponentHolder.init(diskPageSize, diskNumPages, diskMaxOpenFiles);
+        diskBufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+        diskFileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+        memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
+        memBufferCache.open();
+        memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages, new LIFOMetaDataFrameFactory());
+        ioManager = TestStorageManagerComponentHolder.getIOManager();
+        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();
+                }
+            }
+            dir.delete();
+        }
+        memBufferCache.close();
+    }
+
+    public FileReference getInvListsFileRef() {
+        return invIndexFileRef;
+    }
+
+    public int getDiskPageSize() {
+        return diskPageSize;
+    }
+
+    public int getDiskNumPages() {
+        return diskNumPages;
+    }
+
+    public int getDiskMaxOpenFiles() {
+        return diskMaxOpenFiles;
+    }
+
+    public int getMemPageSize() {
+        return memPageSize;
+    }
+
+    public int getMemNumPages() {
+        return memNumPages;
+    }
+
+    public int getHyracksFrameSize() {
+        return hyracksFrameSize;
+    }
+
+    public IOManager getIOManager() {
+        return ioManager;
+    }
+
+    public IBufferCache getDiskBufferCache() {
+        return diskBufferCache;
+    }
+
+    public IFileMapProvider getDiskFileMapProvider() {
+        return diskFileMapProvider;
+    }
+
+    public IInMemoryBufferCache getMemBufferCache() {
+        return memBufferCache;
+    }
+
+    public IInMemoryFreePageManager getMemFreePageManager() {
+        return memFreePageManager;
+    }
+
+    public IHyracksTaskContext getHyracksTastContext() {
+        return ctx;
+    }
+
+    public String getOnDiskDir() {
+        return onDiskDir;
+    }
+
+    public Random getRandom() {
+        return rnd;
+    }
+
+    public ILSMIOOperationScheduler getIOScheduler() {
+        return ioScheduler;
+    }
+
+    public ILSMOperationTrackerFactory getOperationTrackerFactory() {
+        return opTrackerFactory;
+    }
+
+    public ILSMMergePolicy getMergePolicy() {
+        return mergePolicy;
+    }
+
+    public ILSMIOOperationCallbackProvider getIOOperationCallbackProvider() {
+        return ioOpCallbackProvider;
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexDeleteTest.java
new file mode 100644
index 0000000..c71d996
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexDeleteTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexDeleteTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class InMemoryInvertedIndexDeleteTest extends AbstractInvertedIndexDeleteTest {
+    
+    public InMemoryInvertedIndexDeleteTest() {
+        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/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
new file mode 100644
index 0000000..d2b883d
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexInsertTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+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 InMemoryInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
+    
+    public InMemoryInvertedIndexInsertTest() {
+        super(InvertedIndexType.INMEMORY, false, 1);
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexSearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexSearchTest.java
new file mode 100644
index 0000000..f3b3026
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexSearchTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexSearchTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class InMemoryInvertedIndexSearchTest extends AbstractInvertedIndexSearchTest {
+
+    public InMemoryInvertedIndexSearchTest() {
+        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/PartitionedInMemoryInvertedIndexDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexDeleteTest.java
new file mode 100644
index 0000000..eac7765
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexDeleteTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexDeleteTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedInMemoryInvertedIndexDeleteTest extends AbstractInvertedIndexDeleteTest {
+    
+    public PartitionedInMemoryInvertedIndexDeleteTest() {
+        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/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
new file mode 100644
index 0000000..8342efd
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexInsertTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+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 PartitionedInMemoryInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
+
+    public PartitionedInMemoryInvertedIndexInsertTest() {
+        super(InvertedIndexType.PARTITIONED_INMEMORY, false, 1);
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexSearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexSearchTest.java
new file mode 100644
index 0000000..385d65d
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexSearchTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.inmemory;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexSearchTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedInMemoryInvertedIndexSearchTest extends AbstractInvertedIndexSearchTest {
+
+    public PartitionedInMemoryInvertedIndexSearchTest() {
+        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/LSMInvertedIndexMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexMultiThreadTest.java
new file mode 100644
index 0000000..bd48068
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexMultiThreadTest.java
@@ -0,0 +1,162 @@
+/*
+ * Copyright 2009-2012 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.multithread;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
+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.datagen.ProbabilityHelper;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleGenerator;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils;
+
+public class LSMInvertedIndexMultiThreadTest {
+
+    protected final Logger LOGGER = Logger.getLogger(LSMInvertedIndexMultiThreadTest.class.getName());
+
+    // Machine-specific number of threads to use for testing.
+    protected final int REGULAR_NUM_THREADS = Runtime.getRuntime().availableProcessors();
+    // Excessive number of threads for testing.
+    protected final int EXCESSIVE_NUM_THREADS = Runtime.getRuntime().availableProcessors() * 4;
+    protected final int NUM_OPERATIONS = AccessMethodTestsConfig.LSM_INVINDEX_MULTITHREAD_NUM_OPERATIONS;
+
+    protected final LSMInvertedIndexTestHarness harness = new LSMInvertedIndexTestHarness();
+    protected final LSMInvertedIndexWorkerFactory workerFactory = new LSMInvertedIndexWorkerFactory();
+    protected final ArrayList<TestWorkloadConf> workloadConfs = getTestWorkloadConf();
+
+    protected void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    protected void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen, int numThreads,
+            TestWorkloadConf conf, String dataMsg) throws InterruptedException, TreeIndexException, HyracksException {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("LSMInvertedIndex MultiThread Test:\nData: " + dataMsg + "; Threads: " + numThreads
+                    + "; Workload: " + conf.toString() + ".");
+        }
+
+        // 4 batches per thread.
+        int batchSize = (NUM_OPERATIONS / numThreads) / 4;
+
+        LSMInvertedIndexMultiThreadTestDriver driver = new LSMInvertedIndexMultiThreadTestDriver(testCtx.getIndex(),
+                workerFactory, tupleGen.getFieldSerdes(), tupleGen.getFieldGens(), conf.ops, conf.opProbs);
+        driver.init();
+        long[] times = driver.run(numThreads, 1, NUM_OPERATIONS, batchSize);
+        testCtx.getIndex().validate();
+        driver.deinit();
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("LSMInvertedIndex MultiThread Test Time: " + times[0] + "ms");
+        }
+    }
+
+    protected ArrayList<TestWorkloadConf> getTestWorkloadConf() {
+        ArrayList<TestWorkloadConf> workloadConfs = new ArrayList<TestWorkloadConf>();
+
+        // Insert only workload.
+        TestOperation[] insertOnlyOps = new TestOperation[] { TestOperation.INSERT };
+        workloadConfs.add(new TestWorkloadConf(insertOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertOnlyOps.length)));
+
+        // Insert and merge workload.
+        TestOperation[] insertMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(insertMergeOps, ProbabilityHelper
+                .getUniformProbDist(insertMergeOps.length)));
+
+        // Inserts mixed with point searches and scans.
+        TestOperation[] insertSearchOnlyOps = new TestOperation[] { TestOperation.INSERT, TestOperation.POINT_SEARCH,
+                TestOperation.SCAN };
+        workloadConfs.add(new TestWorkloadConf(insertSearchOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertSearchOnlyOps.length)));
+
+        // Inserts, and deletes.
+        TestOperation[] insertDeleteUpdateOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE };
+        workloadConfs.add(new TestWorkloadConf(insertDeleteUpdateOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteUpdateOps.length)));
+
+        // Inserts, deletes and merges.
+        TestOperation[] insertDeleteUpdateMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(insertDeleteUpdateMergeOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteUpdateMergeOps.length)));
+
+        // All operations except merge.
+        TestOperation[] allNoMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.POINT_SEARCH, TestOperation.SCAN };
+        workloadConfs.add(new TestWorkloadConf(allNoMergeOps, ProbabilityHelper
+                .getUniformProbDist(allNoMergeOps.length)));
+
+        // All operations.
+        TestOperation[] allOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.POINT_SEARCH, TestOperation.SCAN, TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(allOps, ProbabilityHelper.getUniformProbDist(allOps.length)));
+
+        return workloadConfs;
+    }
+
+    @Test
+    public void wordTokensInvIndexTest() throws IOException, IndexException, InterruptedException {
+        String dataMsg = "Documents";
+        int[] numThreads = new int[] { REGULAR_NUM_THREADS, EXCESSIVE_NUM_THREADS };
+        for (int i = 0; i < numThreads.length; i++) {
+            for (TestWorkloadConf conf : workloadConfs) {
+                setUp();
+                LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createWordInvIndexTestContext(harness,
+                        getIndexType());
+                TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createStringDocumentTupleGen(harness.getRandom());
+                runTest(testCtx, tupleGen, numThreads[i], conf, dataMsg);
+                tearDown();
+            }
+        }
+    }
+
+    @Test
+    public void hashedNGramTokensInvIndexTest() throws IOException, IndexException, InterruptedException {
+        String dataMsg = "Person Names";
+        int[] numThreads = new int[] { REGULAR_NUM_THREADS, EXCESSIVE_NUM_THREADS };
+        for (int i = 0; i < numThreads.length; i++) {
+            for (TestWorkloadConf conf : workloadConfs) {
+                setUp();
+                LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createHashedNGramInvIndexTestContext(
+                        harness, getIndexType());
+                TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createPersonNamesTupleGen(harness.getRandom());
+                runTest(testCtx, tupleGen, numThreads[i], conf, dataMsg);
+                tearDown();
+            }
+        }
+    }
+
+    protected InvertedIndexType getIndexType() {
+        return InvertedIndexType.LSM;
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexMultiThreadTestDriver.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexMultiThreadTestDriver.java
new file mode 100644
index 0000000..6159969
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexMultiThreadTestDriver.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2012 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.multithread;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.IndexMultiThreadTestDriver;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+
+@SuppressWarnings("rawtypes")
+public class LSMInvertedIndexMultiThreadTestDriver extends IndexMultiThreadTestDriver {
+
+    protected final IFieldValueGenerator[] fieldGens;
+
+    public LSMInvertedIndexMultiThreadTestDriver(IIndex index, IIndexTestWorkerFactory workerFactory,
+            ISerializerDeserializer[] fieldSerdes, IFieldValueGenerator[] fieldGens, TestOperation[] ops,
+            double[] opProbs) {
+        super(index, workerFactory, fieldSerdes, ops, opProbs);
+        this.fieldGens = fieldGens;
+    }
+
+    public DataGenThread createDatagenThread(int numThreads, int numBatches, int batchSize) {
+        return new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, fieldGens, RANDOM_SEED, 2 * numThreads);
+    }
+}
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
new file mode 100644
index 0000000..d6bb3d3
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright 2009-2012 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.multithread;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+
+public class LSMInvertedIndexTestWorker extends AbstractIndexTestWorker {
+
+    protected final LSMInvertedIndex invIndex;
+    protected final List<ITupleReference> documentCorpus = new ArrayList<ITupleReference>();
+    protected final Random rnd = new Random(50);
+
+    protected final IInvertedIndexSearchModifier[] TEST_SEARCH_MODIFIERS = new IInvertedIndexSearchModifier[] {
+            new ConjunctiveSearchModifier(), new JaccardSearchModifier(0.8f), new JaccardSearchModifier(0.5f) };
+
+    public LSMInvertedIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+            int numBatches) {
+        super(dataGen, opSelector, index, numBatches);
+        invIndex = (LSMInvertedIndex) index;
+    }
+
+    @Override
+    public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
+        LSMInvertedIndexAccessor accessor = (LSMInvertedIndexAccessor) indexAccessor;
+        IIndexCursor searchCursor = accessor.createSearchCursor();
+        IIndexCursor rangeSearchCursor = accessor.createRangeSearchCursor();
+        RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+        IBinaryTokenizerFactory tokenizerFactory = invIndex.getTokenizerFactory();
+        int searchModifierIndex = Math.abs(rnd.nextInt()) % TEST_SEARCH_MODIFIERS.length;
+        InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizerFactory.createTokenizer(),
+                TEST_SEARCH_MODIFIERS[searchModifierIndex]);
+
+        switch (op) {
+            case INSERT: {
+                insert(accessor, tuple);
+                break;
+            }
+
+            case DELETE: {
+                // Randomly pick a document from the corpus to delete.
+                if (!documentCorpus.isEmpty()) {
+                    int docIndex = Math.abs(rnd.nextInt()) % documentCorpus.size();
+                    ITupleReference deleteTuple = documentCorpus.get(docIndex);
+                    accessor.delete(deleteTuple);
+                    // Swap tupleIndex with last element.
+                    documentCorpus.set(docIndex, documentCorpus.get(documentCorpus.size() - 1));
+                    documentCorpus.remove(documentCorpus.size() - 1);
+                } else {
+                    // No existing documents to delete, treat this case as an insert.
+                    insert(accessor, tuple);
+                }
+                break;
+            }
+
+            case POINT_SEARCH: {
+                searchCursor.reset();
+                searchPred.setQueryTuple(tuple);
+                searchPred.setQueryFieldIndex(0);
+                try {
+                    accessor.search(searchCursor, searchPred);
+                    consumeCursorTuples(searchCursor);
+                } catch (OccurrenceThresholdPanicException e) {
+                    // Ignore.
+                }
+                break;
+            }
+
+            case SCAN: {
+                rangeSearchCursor.reset();
+                accessor.rangeSearch(rangeSearchCursor, rangePred);
+                consumeCursorTuples(rangeSearchCursor);
+                break;
+            }
+
+            case MERGE: {
+                accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+                break;
+            }
+
+            default:
+                throw new HyracksDataException("Op " + op.toString() + " not supported.");
+        }
+    }
+
+    private void insert(LSMInvertedIndexAccessor accessor, ITupleReference tuple) throws HyracksDataException,
+            IndexException {
+        // Ignore ongoing merges. Do an insert instead.
+        accessor.insert(tuple);
+        // Add tuple to document corpus so we can delete it.
+        ITupleReference copyTuple = TupleUtils.copyTuple(tuple);
+        documentCorpus.add(copyTuple);
+    }
+}
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
new file mode 100644
index 0000000..cce843b
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexWorkerFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2012 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.multithread;
+
+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;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+
+public class LSMInvertedIndexWorkerFactory implements IIndexTestWorkerFactory {
+    @Override
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
+            IIndex index, int numBatches) {
+        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/multithread/PartitionedLSMInvertedIndexMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/PartitionedLSMInvertedIndexMultiThreadTest.java
new file mode 100644
index 0000000..1adaf61
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/PartitionedLSMInvertedIndexMultiThreadTest.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2012 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.multithread;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexMultiThreadTest extends LSMInvertedIndexMultiThreadTest {
+
+    protected InvertedIndexType getIndexType() {
+        return InvertedIndexType.PARTITIONED_LSM;
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleTest.java
new file mode 100644
index 0000000..3059062
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleTest.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Random;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+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.lsm.invertedindex.ondisk.FixedSizeFrameTupleAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeFrameTupleAppender;
+
+public class FixedSizeFrameTupleTest {
+
+    private static int FRAME_SIZE = 4096;
+
+    private Random rnd = new Random(50);
+
+    /**
+     * This test verifies the correct behavior of the FixedSizeFrameTuple class.
+     * Frames containing FixedSizeFrameTuple's require neither tuple slots nor
+     * field slots. The tests inserts generated data into a frame until the
+     * frame is full, and then verifies the frame's contents.
+     * 
+     */
+    @Test
+    public void singleFieldTest() throws Exception {
+        ByteBuffer buffer = ByteBuffer.allocate(FRAME_SIZE);
+
+        ITypeTraits[] fields = new ITypeTraits[1];
+        fields[0] = IntegerPointable.TYPE_TRAITS;
+
+        FixedSizeFrameTupleAppender ftapp = new FixedSizeFrameTupleAppender(FRAME_SIZE, fields);
+        FixedSizeFrameTupleAccessor ftacc = new FixedSizeFrameTupleAccessor(FRAME_SIZE, fields);
+
+        boolean frameHasSpace = true;
+
+        ArrayList<Integer> check = new ArrayList<Integer>();
+
+        ftapp.reset(buffer, true);
+        while (frameHasSpace) {
+            int val = rnd.nextInt();
+            frameHasSpace = ftapp.append(val);
+            if (frameHasSpace) {
+                check.add(val);
+                ftapp.incrementTupleCount(1);
+            }
+        }
+
+        ftacc.reset(buffer);
+        for (int i = 0; i < ftacc.getTupleCount(); i++) {
+            int val = IntegerSerializerDeserializer.getInt(ftacc.getBuffer().array(), ftacc.getTupleStartOffset(i));
+            Assert.assertEquals(check.get(i).intValue(), val);
+        }
+    }
+}
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
new file mode 100644
index 0000000..4813615
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexBulkLoadTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.ondisk;
+
+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 OnDiskInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
+
+    public OnDiskInvertedIndexBulkLoadTest() {
+        super(InvertedIndexType.ONDISK, true, 1);
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
new file mode 100644
index 0000000..09432de
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexLifecycleTest.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2009-2010 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.ondisk;
+
+import java.io.File;
+
+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.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.storage.am.common.AbstractIndexLifecycleTest;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+
+public class OnDiskInvertedIndexLifecycleTest extends AbstractIndexLifecycleTest {
+
+    private final LSMInvertedIndexTestHarness harness = new LSMInvertedIndexTestHarness();
+    private ITreeIndexFrame frame = null;
+
+    @Override
+    protected boolean persistentStateExists() throws Exception {
+        return harness.getInvListsFileRef().getFile().exists()
+                && ((OnDiskInvertedIndex) index).getBTree().getFileReference().getFile().exists();
+    }
+
+    @Override
+    protected boolean isEmptyIndex() throws Exception {
+        if (frame == null) {
+            frame = ((OnDiskInvertedIndex) index).getBTree().getLeafFrameFactory().createFrame();
+        }
+        return ((OnDiskInvertedIndex) index).getBTree().isEmptyTree(frame);
+    }
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        ITypeTraits[] tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
+        IBinaryComparatorFactory[] tokenCmpFactories = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
+                .of(UTF8StringPointable.FACTORY) };
+        ITypeTraits[] invListTypeTraits = new ITypeTraits[] { IntegerPointable.TYPE_TRAITS };
+        IBinaryComparatorFactory[] invListCmpFactories = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
+                .of(IntegerPointable.FACTORY) };
+        IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
+        FileReference btreeFile = new FileReference(new File(harness.getInvListsFileRef().getFile().getPath() + "_btree"));
+        index = new OnDiskInvertedIndex(harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), invListBuilder,
+                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, harness.getInvListsFileRef(),
+                btreeFile);
+
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        index.deactivate();
+        index.destroy();
+        harness.tearDown();
+    }
+
+    @Override
+    protected void performInsertions() throws Exception {
+        // Do nothing.
+    }
+
+    @Override
+    protected void checkInsertions() throws Exception {
+        // Do nothing.
+    }
+
+    @Override
+    protected void clearCheckableInsertions() throws Exception {
+        // Do nothing.
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchTest.java
new file mode 100644
index 0000000..76d9200
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.ondisk;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexSearchTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class OnDiskInvertedIndexSearchTest extends AbstractInvertedIndexSearchTest {
+
+    public OnDiskInvertedIndexSearchTest() {
+        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
new file mode 100644
index 0000000..f641630
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexBulkLoadTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.ondisk;
+
+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 PartitionedOnDiskInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
+
+    public PartitionedOnDiskInvertedIndexBulkLoadTest() {
+        super(InvertedIndexType.PARTITIONED_ONDISK, true, 1);
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexSearchTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexSearchTest.java
new file mode 100644
index 0000000..4fa25ed
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexSearchTest.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2012 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.ondisk;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexSearchTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedOnDiskInvertedIndexSearchTest extends AbstractInvertedIndexSearchTest {
+
+    public PartitionedOnDiskInvertedIndexSearchTest() {
+        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/tokenizers/NGramTokenizerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramTokenizerTest.java
new file mode 100644
index 0000000..33ea4f5
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramTokenizerTest.java
@@ -0,0 +1,228 @@
+/*
+ * Copyright 2009-2010 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.tokenizers;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
+public class NGramTokenizerTest {
+
+    private char PRECHAR = '#';
+    private char POSTCHAR = '$';
+
+    private String str = "Jürgen S. Generic's Car";
+    private byte[] inputBuffer;
+
+    private int gramLength = 3;
+
+    private void getExpectedGrams(String s, int gramLength, ArrayList<String> grams, boolean prePost) {
+
+        String tmp = s.toLowerCase();
+        if (prePost) {
+            StringBuilder preBuilder = new StringBuilder();
+            for (int i = 0; i < gramLength - 1; i++) {
+                preBuilder.append(PRECHAR);
+            }
+            String pre = preBuilder.toString();
+
+            StringBuilder postBuilder = new StringBuilder();
+            for (int i = 0; i < gramLength - 1; i++) {
+                postBuilder.append(POSTCHAR);
+            }
+            String post = postBuilder.toString();
+
+            tmp = pre + s.toLowerCase() + post;
+        }
+
+        for (int i = 0; i < tmp.length() - gramLength + 1; i++) {
+            String gram = tmp.substring(i, i + gramLength);
+            grams.add(gram);
+        }
+    }
+
+    @Before
+    public void init() throws Exception {
+        // serialize string into bytes
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutput dos = new DataOutputStream(baos);
+        dos.writeUTF(str);
+        inputBuffer = baos.toByteArray();
+    }
+
+    void runTestNGramTokenizerWithCountedHashedUTF8Tokens(boolean prePost) throws IOException {
+        HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
+        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, false,
+                false, tokenFactory);
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+
+        ArrayList<String> expectedGrams = new ArrayList<String>();
+        getExpectedGrams(str, gramLength, expectedGrams, prePost);
+        ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
+        HashMap<String, Integer> gramCounts = new HashMap<String, Integer>();
+        for (String s : expectedGrams) {
+            Integer count = gramCounts.get(s);
+            if (count == null) {
+                count = 1;
+                gramCounts.put(s, count);
+            } else {
+                count++;
+            }
+
+            int hash = tokenHash(s, count);
+            expectedHashedGrams.add(hash);
+        }
+
+        int tokenCount = 0;
+
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+
+            // serialize hashed token
+            GrowableArray tokenData = new GrowableArray();
+
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenData);
+
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenData.getByteArray());
+            DataInput in = new DataInputStream(bais);
+
+            Integer hashedGram = in.readInt();
+
+            // System.out.println(hashedGram);
+
+            Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
+
+            tokenCount++;
+        }
+        // System.out.println("---------");
+    }
+
+    void runTestNGramTokenizerWithHashedUTF8Tokens(boolean prePost) throws IOException {
+        HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
+        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
+                tokenFactory);
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+
+        ArrayList<String> expectedGrams = new ArrayList<String>();
+        getExpectedGrams(str, gramLength, expectedGrams, prePost);
+        ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
+        for (String s : expectedGrams) {
+            int hash = tokenHash(s, 1);
+            expectedHashedGrams.add(hash);
+        }
+
+        int tokenCount = 0;
+
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+
+            // serialize hashed token
+            GrowableArray tokenData = new GrowableArray();
+
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenData);
+
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenData.getByteArray());
+            DataInput in = new DataInputStream(bais);
+
+            Integer hashedGram = in.readInt();
+
+            // System.out.println(hashedGram);
+
+            Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
+
+            tokenCount++;
+        }
+        // System.out.println("---------");
+    }
+
+    void runTestNGramTokenizerWithUTF8Tokens(boolean prePost) throws IOException {
+        UTF8NGramTokenFactory tokenFactory = new UTF8NGramTokenFactory();
+        NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
+                tokenFactory);
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+
+        ArrayList<String> expectedGrams = new ArrayList<String>();
+        getExpectedGrams(str, gramLength, expectedGrams, prePost);
+
+        int tokenCount = 0;
+
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+
+            // serialize hashed token
+            GrowableArray tokenData = new GrowableArray();
+
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenData);
+
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenData.getByteArray());
+            DataInput in = new DataInputStream(bais);
+
+            String strGram = in.readUTF();
+
+            // System.out.println("\"" + strGram + "\"");
+
+            Assert.assertEquals(expectedGrams.get(tokenCount), strGram);
+
+            tokenCount++;
+        }
+        // System.out.println("---------");
+    }
+
+    @Test
+    public void testNGramTokenizerWithCountedHashedUTF8Tokens() throws Exception {
+        runTestNGramTokenizerWithCountedHashedUTF8Tokens(false);
+        runTestNGramTokenizerWithCountedHashedUTF8Tokens(true);
+    }
+
+    @Test
+    public void testNGramTokenizerWithHashedUTF8Tokens() throws Exception {
+        runTestNGramTokenizerWithHashedUTF8Tokens(false);
+        runTestNGramTokenizerWithHashedUTF8Tokens(true);
+    }
+
+    @Test
+    public void testNGramTokenizerWithUTF8Tokens() throws IOException {
+        runTestNGramTokenizerWithUTF8Tokens(false);
+        runTestNGramTokenizerWithUTF8Tokens(true);
+    }
+
+    public int tokenHash(String token, int tokenCount) {
+        int h = AbstractUTF8Token.GOLDEN_RATIO_32;
+        for (int i = 0; i < token.length(); i++) {
+            h ^= token.charAt(i);
+            h *= AbstractUTF8Token.GOLDEN_RATIO_32;
+        }
+        return h + tokenCount;
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/WordTokenizerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/WordTokenizerTest.java
new file mode 100644
index 0000000..3ff9304
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/tokenizers/WordTokenizerTest.java
@@ -0,0 +1,210 @@
+/*
+ * Copyright 2009-2010 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.tokenizers;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import junit.framework.Assert;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
+public class WordTokenizerTest {
+
+    private String text = "Hello World, I would like to inform you of the importance of Foo Bar. Yes, Foo Bar. Jürgen.";
+    private byte[] inputBuffer;
+
+    private ArrayList<String> expectedUTF8Tokens = new ArrayList<String>();
+    private ArrayList<Integer> expectedHashedUTF8Tokens = new ArrayList<Integer>();
+    private ArrayList<Integer> expectedCountedHashedUTF8Tokens = new ArrayList<Integer>();
+
+    private boolean isSeparator(char c) {
+        return !(Character.isLetterOrDigit(c) || Character.getType(c) == Character.OTHER_LETTER || Character.getType(c) == Character.OTHER_NUMBER);
+    }
+
+    private void tokenize(String text, ArrayList<String> tokens) {
+        String lowerCaseText = text.toLowerCase();
+        int startIx = 0;
+
+        // Skip separators at beginning of string.
+        while (isSeparator(lowerCaseText.charAt(startIx))) {
+            startIx++;
+        }
+        while (startIx < lowerCaseText.length()) {
+            while (startIx < lowerCaseText.length() && isSeparator(lowerCaseText.charAt(startIx))) {
+                startIx++;
+            }
+            int tokenStart = startIx;
+
+            while (startIx < lowerCaseText.length() && !isSeparator(lowerCaseText.charAt(startIx))) {
+                startIx++;
+            }
+            int tokenEnd = startIx;
+
+            // Emit token.
+            String token = lowerCaseText.substring(tokenStart, tokenEnd);
+
+            tokens.add(token);
+        }
+    }
+
+    @Before
+    public void init() throws IOException {
+        // serialize text into bytes
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutput dos = new DataOutputStream(baos);
+        dos.writeUTF(text);
+        inputBuffer = baos.toByteArray();
+
+        // init expected string tokens
+        tokenize(text, expectedUTF8Tokens);
+
+        // hashed tokens ignoring token count
+        for (int i = 0; i < expectedUTF8Tokens.size(); i++) {
+            int hash = tokenHash(expectedUTF8Tokens.get(i), 1);
+            expectedHashedUTF8Tokens.add(hash);
+        }
+
+        // hashed tokens using token count
+        HashMap<String, Integer> tokenCounts = new HashMap<String, Integer>();
+        for (int i = 0; i < expectedUTF8Tokens.size(); i++) {
+            Integer count = tokenCounts.get(expectedUTF8Tokens.get(i));
+            if (count == null) {
+                count = 1;
+                tokenCounts.put(expectedUTF8Tokens.get(i), count);
+            } else {
+                count++;
+            }
+
+            int hash = tokenHash(expectedUTF8Tokens.get(i), count);
+            expectedCountedHashedUTF8Tokens.add(hash);
+        }
+    }
+
+    @Test
+    public void testWordTokenizerWithCountedHashedUTF8Tokens() throws IOException {
+
+        HashedUTF8WordTokenFactory tokenFactory = new HashedUTF8WordTokenFactory();
+        DelimitedUTF8StringBinaryTokenizer tokenizer = new DelimitedUTF8StringBinaryTokenizer(false, false,
+                tokenFactory);
+
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+
+        int tokenCount = 0;
+
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+
+            // serialize hashed token
+            GrowableArray tokenData = new GrowableArray();
+
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenData);
+
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenData.getByteArray());
+            DataInput in = new DataInputStream(bais);
+
+            Integer hashedToken = in.readInt();
+
+            Assert.assertEquals(hashedToken, expectedCountedHashedUTF8Tokens.get(tokenCount));
+
+            tokenCount++;
+        }
+    }
+
+    @Test
+    public void testWordTokenizerWithHashedUTF8Tokens() throws IOException {
+
+        HashedUTF8WordTokenFactory tokenFactory = new HashedUTF8WordTokenFactory();
+        DelimitedUTF8StringBinaryTokenizer tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, false, tokenFactory);
+
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+
+        int tokenCount = 0;
+
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+
+            // serialize hashed token
+            GrowableArray tokenData = new GrowableArray();
+
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenData);
+
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenData.getByteArray());
+            DataInput in = new DataInputStream(bais);
+
+            Integer hashedToken = in.readInt();
+
+            Assert.assertEquals(expectedHashedUTF8Tokens.get(tokenCount), hashedToken);
+
+            tokenCount++;
+        }
+    }
+
+    @Test
+    public void testWordTokenizerWithUTF8Tokens() throws IOException {
+
+        UTF8WordTokenFactory tokenFactory = new UTF8WordTokenFactory();
+        DelimitedUTF8StringBinaryTokenizer tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, false, tokenFactory);
+
+        tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+
+        int tokenCount = 0;
+
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+
+            // serialize hashed token
+            GrowableArray tokenData = new GrowableArray();
+
+            IToken token = tokenizer.getToken();
+            token.serializeToken(tokenData);
+
+            // deserialize token
+            ByteArrayInputStream bais = new ByteArrayInputStream(tokenData.getByteArray());
+            DataInput in = new DataInputStream(bais);
+
+            String strToken = in.readUTF();
+
+            Assert.assertEquals(expectedUTF8Tokens.get(tokenCount), strToken);
+
+            tokenCount++;
+        }
+    }
+
+    // JAQL Hash
+    public int tokenHash(String token, int tokenCount) {
+        int h = AbstractUTF8Token.GOLDEN_RATIO_32;
+        for (int i = 0; i < token.length(); i++) {
+            h ^= token.charAt(i);
+            h *= AbstractUTF8Token.GOLDEN_RATIO_32;
+        }
+        return h + tokenCount;
+    }
+}
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
new file mode 100644
index 0000000..870e6d9
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -0,0 +1,249 @@
+/*
+ * Copyright 2009-2010 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.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.InvertedIndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+
+@SuppressWarnings("rawtypes")
+public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
+
+    public static enum InvertedIndexType {
+        INMEMORY,
+        ONDISK,
+        LSM,
+        PARTITIONED_INMEMORY,
+        PARTITIONED_ONDISK,
+        PARTITIONED_LSM
+    };
+
+    protected IInvertedIndex invIndex;
+    protected IBinaryComparatorFactory[] allCmpFactories;
+    protected IBinaryTokenizerFactory tokenizerFactory;
+    protected InvertedIndexType invIndexType;
+    protected InvertedIndexTokenizingTupleIterator indexTupleIter;
+    protected HashSet<Comparable> allTokens = new HashSet<Comparable>();
+    protected List<ITupleReference> documentCorpus = new ArrayList<ITupleReference>();
+
+    public LSMInvertedIndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index,
+            IBinaryTokenizerFactory tokenizerFactory, InvertedIndexType invIndexType,
+            InvertedIndexTokenizingTupleIterator indexTupleIter) {
+        super(fieldSerdes, index);
+        invIndex = (IInvertedIndex) index;
+        this.tokenizerFactory = tokenizerFactory;
+        this.invIndexType = invIndexType;
+        this.indexTupleIter = indexTupleIter;
+    }
+
+    @Override
+    public int getKeyFieldCount() {
+        return fieldSerdes.length;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getComparatorFactories() {
+        if (allCmpFactories == null) {
+            // Concatenate token and inv-list comparators.
+            IInvertedIndex invIndex = (IInvertedIndex) index;
+            IBinaryComparatorFactory[] tokenCmpFactories = invIndex.getTokenCmpFactories();
+            IBinaryComparatorFactory[] invListCmpFactories = invIndex.getInvListCmpFactories();
+            int totalCmpCount = tokenCmpFactories.length + invListCmpFactories.length;
+            allCmpFactories = new IBinaryComparatorFactory[totalCmpCount];
+            for (int i = 0; i < tokenCmpFactories.length; i++) {
+                allCmpFactories[i] = tokenCmpFactories[i];
+            }
+            for (int i = 0; i < invListCmpFactories.length; i++) {
+                allCmpFactories[i + tokenCmpFactories.length] = invListCmpFactories[i];
+            }
+        }
+        return allCmpFactories;
+    }
+
+    public static LSMInvertedIndexTestContext create(LSMInvertedIndexTestHarness harness,
+            ISerializerDeserializer[] fieldSerdes, int tokenFieldCount, IBinaryTokenizerFactory tokenizerFactory,
+            InvertedIndexType invIndexType) throws IndexException {
+        ITypeTraits[] allTypeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
+        IBinaryComparatorFactory[] allCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
+                fieldSerdes.length);
+        // Set token type traits and comparators.
+        ITypeTraits[] tokenTypeTraits = new ITypeTraits[tokenFieldCount];
+        IBinaryComparatorFactory[] tokenCmpFactories = new IBinaryComparatorFactory[tokenFieldCount];
+        for (int i = 0; i < tokenTypeTraits.length; i++) {
+            tokenTypeTraits[i] = allTypeTraits[i];
+            tokenCmpFactories[i] = allCmpFactories[i];
+        }
+        // Set inverted-list element type traits and comparators.
+        int invListFieldCount = fieldSerdes.length - tokenFieldCount;
+        ITypeTraits[] invListTypeTraits = new ITypeTraits[invListFieldCount];
+        IBinaryComparatorFactory[] invListCmpFactories = new IBinaryComparatorFactory[invListFieldCount];
+        for (int i = 0; i < invListTypeTraits.length; i++) {
+            invListTypeTraits[i] = allTypeTraits[i + tokenFieldCount];
+            invListCmpFactories[i] = allCmpFactories[i + tokenFieldCount];
+        }
+        // Create index and test context.        
+        IInvertedIndex invIndex;
+        switch (invIndexType) {
+            case INMEMORY: {
+                invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getMemBufferCache(),
+                        harness.getMemFreePageManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                        tokenCmpFactories, tokenizerFactory);
+                break;
+            }
+            case PARTITIONED_INMEMORY: {
+                invIndex = InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(harness.getMemBufferCache(),
+                        harness.getMemFreePageManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                        tokenCmpFactories, tokenizerFactory);
+                break;
+            }
+            case ONDISK: {
+                invIndex = InvertedIndexUtils.createOnDiskInvertedIndex(harness.getDiskBufferCache(),
+                        harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                        tokenCmpFactories, harness.getInvListsFileRef());
+                break;
+            }
+            case PARTITIONED_ONDISK: {
+                invIndex = InvertedIndexUtils.createPartitionedOnDiskInvertedIndex(harness.getDiskBufferCache(),
+                        harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                        tokenCmpFactories, harness.getInvListsFileRef());
+                break;
+            }
+            case LSM: {
+                invIndex = InvertedIndexUtils.createLSMInvertedIndex(harness.getMemBufferCache(),
+                        harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
+                        invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                        harness.getDiskBufferCache(), harness.getIOManager(), harness.getOnDiskDir(),
+                        harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                        harness.getIOOperationCallbackProvider());
+                break;
+            }
+            case PARTITIONED_LSM: {
+                invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(harness.getMemBufferCache(),
+                        harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
+                        invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                        harness.getDiskBufferCache(), harness.getIOManager(), harness.getOnDiskDir(),
+                        harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                        harness.getIOOperationCallbackProvider());
+                break;
+            }
+            default: {
+                throw new InvertedIndexException("Unknow inverted-index type '" + invIndexType + "'.");
+            }
+        }
+        InvertedIndexTokenizingTupleIterator indexTupleIter = null;
+        switch (invIndexType) {
+            case INMEMORY:
+            case ONDISK:
+            case LSM: {
+                indexTupleIter = new InvertedIndexTokenizingTupleIterator(invIndex.getTokenTypeTraits().length,
+                        invIndex.getInvListTypeTraits().length, tokenizerFactory.createTokenizer());
+                break;
+            }
+            case PARTITIONED_INMEMORY:
+            case PARTITIONED_ONDISK:
+            case PARTITIONED_LSM: {
+                indexTupleIter = new PartitionedInvertedIndexTokenizingTupleIterator(
+                        invIndex.getTokenTypeTraits().length, invIndex.getInvListTypeTraits().length,
+                        tokenizerFactory.createTokenizer());
+                break;
+            }
+            default: {
+                throw new InvertedIndexException("Unknow inverted-index type '" + invIndexType + "'.");
+            }
+        }
+        LSMInvertedIndexTestContext testCtx = new LSMInvertedIndexTestContext(fieldSerdes, invIndex, tokenizerFactory,
+                invIndexType, indexTupleIter);
+        return testCtx;
+    }
+
+    public void insertCheckTuples(ITupleReference tuple, Collection<CheckTuple> checkTuples)
+            throws HyracksDataException {
+        documentCorpus.add(TupleUtils.copyTuple(tuple));
+        indexTupleIter.reset(tuple);
+        while (indexTupleIter.hasNext()) {
+            indexTupleIter.next();
+            ITupleReference insertTuple = indexTupleIter.getTuple();
+            CheckTuple checkTuple = createCheckTuple(insertTuple);
+            insertCheckTuple(checkTuple, checkTuples);
+            allTokens.add(checkTuple.getField(0));
+        }
+    }
+
+    public void deleteCheckTuples(ITupleReference tuple, Collection<CheckTuple> checkTuples)
+            throws HyracksDataException {
+        indexTupleIter.reset(tuple);
+        while (indexTupleIter.hasNext()) {
+            indexTupleIter.next();
+            ITupleReference insertTuple = indexTupleIter.getTuple();
+            CheckTuple checkTuple = createCheckTuple(insertTuple);
+            deleteCheckTuple(checkTuple, checkTuples);
+        }
+    }
+
+    public HashSet<Comparable> getAllTokens() {
+        return allTokens;
+    }
+
+    @SuppressWarnings("unchecked")
+    public CheckTuple createCheckTuple(ITupleReference tuple) throws HyracksDataException {
+        CheckTuple checkTuple = new CheckTuple(fieldSerdes.length, fieldSerdes.length);
+        for (int i = 0; i < fieldSerdes.length; i++) {
+            ByteArrayInputStream bains = new ByteArrayInputStream(tuple.getFieldData(i), tuple.getFieldStart(i),
+                    tuple.getFieldLength(i));
+            DataInput in = new DataInputStream(bains);
+            Comparable field = (Comparable) fieldSerdes[i].deserialize(in);
+            checkTuple.appendField(field);
+        }
+        return checkTuple;
+    }
+
+    @Override
+    public void upsertCheckTuple(CheckTuple checkTuple, Collection<CheckTuple> checkTuples) {
+        throw new UnsupportedOperationException("Upsert not supported by inverted index.");
+    }
+
+    public IBinaryTokenizerFactory getTokenizerFactory() {
+        return tokenizerFactory;
+    }
+
+    public List<ITupleReference> getDocumentCorpus() {
+        return documentCorpus;
+    }
+
+    public InvertedIndexType getInvertedIndexType() {
+        return invIndexType;
+    }
+}
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
new file mode 100644
index 0000000..97f78f3
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -0,0 +1,568 @@
+/*
+ * Copyright 2009-2010 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.util;
+
+import static org.junit.Assert.fail;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestUtils;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DocumentStringFieldValueGenerator;
+import edu.uci.ics.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+import edu.uci.ics.hyracks.storage.am.common.datagen.PersonNameFieldValueGenerator;
+import edu.uci.ics.hyracks.storage.am.common.datagen.SortedIntegerFieldValueGenerator;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleGenerator;
+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.PermutingTupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+@SuppressWarnings("rawtypes")
+public class LSMInvertedIndexTestUtils {
+
+    public static final int TEST_GRAM_LENGTH = 3;
+
+    public static TupleGenerator createStringDocumentTupleGen(Random rnd) throws IOException {
+        IFieldValueGenerator[] fieldGens = new IFieldValueGenerator[2];
+        fieldGens[0] = new DocumentStringFieldValueGenerator(2, 10, 10000, rnd);
+        fieldGens[1] = new SortedIntegerFieldValueGenerator(0);
+        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+        TupleGenerator tupleGen = new TupleGenerator(fieldGens, fieldSerdes, 0);
+        return tupleGen;
+    }
+
+    public static TupleGenerator createPersonNamesTupleGen(Random rnd) throws IOException {
+        IFieldValueGenerator[] fieldGens = new IFieldValueGenerator[2];
+        fieldGens[0] = new PersonNameFieldValueGenerator(rnd, 0.5f);
+        fieldGens[1] = new SortedIntegerFieldValueGenerator(0);
+        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+        TupleGenerator tupleGen = new TupleGenerator(fieldGens, fieldSerdes, 0);
+        return tupleGen;
+    }
+
+    private static ISerializerDeserializer[] getNonHashedIndexFieldSerdes(InvertedIndexType invIndexType)
+            throws IndexException {
+        ISerializerDeserializer[] fieldSerdes = null;
+        switch (invIndexType) {
+            case INMEMORY:
+            case ONDISK:
+            case LSM: {
+                fieldSerdes = new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE,
+                        IntegerSerializerDeserializer.INSTANCE };
+                break;
+            }
+            case PARTITIONED_INMEMORY:
+            case PARTITIONED_ONDISK:
+            case PARTITIONED_LSM: {
+                // Such indexes also include the set-size for partitioning.
+                fieldSerdes = new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE,
+                        ShortSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+                break;
+            }
+            default: {
+                throw new IndexException("Unhandled inverted index type '" + invIndexType + "'.");
+            }
+        }
+        return fieldSerdes;
+    }
+
+    private static ISerializerDeserializer[] getHashedIndexFieldSerdes(InvertedIndexType invIndexType)
+            throws IndexException {
+        ISerializerDeserializer[] fieldSerdes = null;
+        switch (invIndexType) {
+            case INMEMORY:
+            case ONDISK:
+            case LSM: {
+                fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE,
+                        IntegerSerializerDeserializer.INSTANCE };
+                break;
+            }
+            case PARTITIONED_INMEMORY:
+            case PARTITIONED_ONDISK:
+            case PARTITIONED_LSM: {
+                // Such indexes also include the set-size for partitioning.
+                fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE,
+                        ShortSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+                break;
+            }
+            default: {
+                throw new IndexException("Unhandled inverted index type '" + invIndexType + "'.");
+            }
+        }
+        return fieldSerdes;
+    }
+
+    public static LSMInvertedIndexTestContext createWordInvIndexTestContext(LSMInvertedIndexTestHarness harness,
+            InvertedIndexType invIndexType) throws IOException, IndexException {
+        ISerializerDeserializer[] fieldSerdes = getNonHashedIndexFieldSerdes(invIndexType);
+        ITokenFactory tokenFactory = new UTF8WordTokenFactory();
+        IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
+                tokenFactory);
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
+                fieldSerdes.length - 1, tokenizerFactory, invIndexType);
+        return testCtx;
+    }
+
+    public static LSMInvertedIndexTestContext createHashedWordInvIndexTestContext(LSMInvertedIndexTestHarness harness,
+            InvertedIndexType invIndexType) throws IOException, IndexException {
+        ISerializerDeserializer[] fieldSerdes = getHashedIndexFieldSerdes(invIndexType);
+        ITokenFactory tokenFactory = new HashedUTF8WordTokenFactory();
+        IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
+                tokenFactory);
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
+                fieldSerdes.length - 1, tokenizerFactory, invIndexType);
+        return testCtx;
+    }
+
+    public static LSMInvertedIndexTestContext createNGramInvIndexTestContext(LSMInvertedIndexTestHarness harness,
+            InvertedIndexType invIndexType) throws IOException, IndexException {
+        ISerializerDeserializer[] fieldSerdes = getNonHashedIndexFieldSerdes(invIndexType);
+        ITokenFactory tokenFactory = new UTF8NGramTokenFactory();
+        IBinaryTokenizerFactory tokenizerFactory = new NGramUTF8StringBinaryTokenizerFactory(TEST_GRAM_LENGTH, true,
+                true, false, tokenFactory);
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
+                fieldSerdes.length - 1, tokenizerFactory, invIndexType);
+        return testCtx;
+    }
+
+    public static LSMInvertedIndexTestContext createHashedNGramInvIndexTestContext(LSMInvertedIndexTestHarness harness,
+            InvertedIndexType invIndexType) throws IOException, IndexException {
+        ISerializerDeserializer[] fieldSerdes = getHashedIndexFieldSerdes(invIndexType);
+        ITokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
+        IBinaryTokenizerFactory tokenizerFactory = new NGramUTF8StringBinaryTokenizerFactory(TEST_GRAM_LENGTH, true,
+                true, false, tokenFactory);
+        LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
+                fieldSerdes.length - 1, tokenizerFactory, invIndexType);
+        return testCtx;
+    }
+
+    public static void bulkLoadInvIndex(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen, int numDocs)
+            throws IndexException, IOException {
+        SortedSet<CheckTuple> tmpMemIndex = new TreeSet<CheckTuple>();
+        // First generate the expected index by inserting the documents one-by-one.
+        for (int i = 0; i < numDocs; i++) {
+            ITupleReference tuple = tupleGen.next();
+            testCtx.insertCheckTuples(tuple, tmpMemIndex);
+        }
+        ISerializerDeserializer[] fieldSerdes = testCtx.getFieldSerdes();
+
+        // Use the expected index to bulk-load the actual index.
+        IIndexBulkLoader bulkLoader = testCtx.getIndex().createBulkLoader(1.0f, false, numDocs);
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(testCtx.getFieldSerdes().length);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        Iterator<CheckTuple> checkTupleIter = tmpMemIndex.iterator();
+        while (checkTupleIter.hasNext()) {
+            CheckTuple checkTuple = checkTupleIter.next();
+            OrderedIndexTestUtils.createTupleFromCheckTuple(checkTuple, tupleBuilder, tuple, fieldSerdes);
+            bulkLoader.add(tuple);
+        }
+        bulkLoader.end();
+
+        // Add all check tuples from the temp index to the text context.
+        testCtx.getCheckTuples().addAll(tmpMemIndex);
+    }
+
+    public static void insertIntoInvIndex(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen, int numDocs)
+            throws IOException, IndexException {
+        // InMemoryInvertedIndex only supports insert.
+        for (int i = 0; i < numDocs; i++) {
+            ITupleReference tuple = tupleGen.next();
+            testCtx.getIndexAccessor().insert(tuple);
+            testCtx.insertCheckTuples(tuple, testCtx.getCheckTuples());
+        }
+    }
+
+    public static void deleteFromInvIndex(LSMInvertedIndexTestContext testCtx, Random rnd, int numDocsToDelete)
+            throws HyracksDataException, IndexException {
+        List<ITupleReference> documentCorpus = testCtx.getDocumentCorpus();
+        for (int i = 0; i < numDocsToDelete && !documentCorpus.isEmpty(); i++) {
+            int size = documentCorpus.size();
+            int tupleIndex = Math.abs(rnd.nextInt()) % size;
+            ITupleReference deleteTuple = documentCorpus.get(tupleIndex);
+            testCtx.getIndexAccessor().delete(deleteTuple);
+            testCtx.deleteCheckTuples(deleteTuple, testCtx.getCheckTuples());
+            // Swap tupleIndex with last element.
+            documentCorpus.set(tupleIndex, documentCorpus.get(size - 1));
+            documentCorpus.remove(size - 1);
+        }
+    }
+
+    /**
+     * Compares actual and expected indexes using the rangeSearch() method of the inverted-index accessor.
+     */
+    public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx)
+            throws HyracksDataException, IndexException {
+        IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
+        int tokenFieldCount = invIndex.getTokenTypeTraits().length;
+        int invListFieldCount = invIndex.getInvListTypeTraits().length;
+        IInvertedIndexAccessor invIndexAccessor = (IInvertedIndexAccessor) invIndex.createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        IIndexCursor invIndexCursor = invIndexAccessor.createRangeSearchCursor();
+        MultiComparator tokenCmp = MultiComparator.create(invIndex.getTokenCmpFactories());
+        IBinaryComparatorFactory[] tupleCmpFactories = new IBinaryComparatorFactory[tokenFieldCount + invListFieldCount];
+        for (int i = 0; i < tokenFieldCount; i++) {
+            tupleCmpFactories[i] = invIndex.getTokenCmpFactories()[i];
+        }
+        for (int i = 0; i < invListFieldCount; i++) {
+            tupleCmpFactories[tokenFieldCount + i] = invIndex.getInvListCmpFactories()[i];
+        }
+        MultiComparator tupleCmp = MultiComparator.create(tupleCmpFactories);
+        RangePredicate nullPred = new RangePredicate(null, null, true, true, tokenCmp, tokenCmp);
+        invIndexAccessor.rangeSearch(invIndexCursor, nullPred);
+
+        // Helpers for generating a serialized inverted-list element from a CheckTuple from the expected index.
+        ISerializerDeserializer[] fieldSerdes = testCtx.getFieldSerdes();
+        ArrayTupleBuilder expectedBuilder = new ArrayTupleBuilder(fieldSerdes.length);
+        ArrayTupleReference expectedTuple = new ArrayTupleReference();
+
+        Iterator<CheckTuple> expectedIter = testCtx.getCheckTuples().iterator();
+
+        // Compare index elements.
+        try {
+            while (invIndexCursor.hasNext() && expectedIter.hasNext()) {
+                invIndexCursor.next();
+                ITupleReference actualTuple = invIndexCursor.getTuple();
+                CheckTuple expected = expectedIter.next();
+                OrderedIndexTestUtils.createTupleFromCheckTuple(expected, expectedBuilder, expectedTuple, fieldSerdes);
+                if (tupleCmp.compare(actualTuple, expectedTuple) != 0) {
+                    fail("Index entries differ for token '" + expected.getField(0) + "'.");
+                }
+            }
+            if (expectedIter.hasNext()) {
+                fail("Indexes do not match. Actual index is missing entries.");
+            }
+            if (invIndexCursor.hasNext()) {
+                fail("Indexes do not match. Actual index contains too many entries.");
+            }
+        } finally {
+            invIndexCursor.close();
+        }
+    }
+
+    /**
+     * Compares actual and expected indexes by comparing their inverted-lists one by one. Exercises the openInvertedListCursor() method of the inverted-index accessor.
+     */
+    @SuppressWarnings("unchecked")
+    public static void compareActualAndExpectedIndexes(LSMInvertedIndexTestContext testCtx)
+            throws HyracksDataException, IndexException {
+        IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
+        ISerializerDeserializer[] fieldSerdes = testCtx.getFieldSerdes();
+        MultiComparator invListCmp = MultiComparator.create(invIndex.getInvListCmpFactories());
+        IInvertedIndexAccessor invIndexAccessor = (IInvertedIndexAccessor) testCtx.getIndexAccessor();
+        int tokenFieldCount = invIndex.getTokenTypeTraits().length;
+        int invListFieldCount = invIndex.getInvListTypeTraits().length;
+        // All tokens that were inserted into the indexes.
+        Iterator<Comparable> tokensIter = testCtx.getAllTokens().iterator();
+
+        // Search key for finding an inverted-list in the actual index.
+        ArrayTupleBuilder searchKeyBuilder = new ArrayTupleBuilder(tokenFieldCount);
+        ArrayTupleReference searchKey = new ArrayTupleReference();
+        // Cursor over inverted list from actual index.
+        IInvertedListCursor actualInvListCursor = invIndexAccessor.createInvertedListCursor();
+
+        // Helpers for generating a serialized inverted-list element from a CheckTuple from the expected index.
+        ArrayTupleBuilder expectedBuilder = new ArrayTupleBuilder(fieldSerdes.length);
+        // Includes the token fields.
+        ArrayTupleReference completeExpectedTuple = new ArrayTupleReference();
+        // Field permutation and permuting tuple reference to strip away token fields from completeExpectedTuple.
+        int[] fieldPermutation = new int[invListFieldCount];
+        for (int i = 0; i < fieldPermutation.length; i++) {
+            fieldPermutation[i] = tokenFieldCount + i;
+        }
+        PermutingTupleReference expectedTuple = new PermutingTupleReference(fieldPermutation);
+
+        // Iterate over all tokens. Find the inverted-lists in actual and expected indexes. Compare the inverted lists,
+        while (tokensIter.hasNext()) {
+            Comparable token = tokensIter.next();
+
+            // Position inverted-list iterator on expected index.
+            CheckTuple checkLowKey = new CheckTuple(tokenFieldCount, tokenFieldCount);
+            checkLowKey.appendField(token);
+            CheckTuple checkHighKey = new CheckTuple(tokenFieldCount, tokenFieldCount);
+            checkHighKey.appendField(token);
+
+            SortedSet<CheckTuple> expectedInvList = OrderedIndexTestUtils.getPrefixExpectedSubset(
+                    testCtx.getCheckTuples(), checkLowKey, checkHighKey);
+            Iterator<CheckTuple> expectedInvListIter = expectedInvList.iterator();
+
+            // Position inverted-list cursor in actual index.
+            OrderedIndexTestUtils.createTupleFromCheckTuple(checkLowKey, searchKeyBuilder, searchKey, fieldSerdes);
+            invIndexAccessor.openInvertedListCursor(actualInvListCursor, searchKey);
+
+            if (actualInvListCursor.size() != expectedInvList.size()) {
+                fail("Actual and expected inverted lists for token '" + token.toString()
+                        + "' have different sizes. Actual size: " + actualInvListCursor.size() + ". Expected size: "
+                        + expectedInvList.size() + ".");
+            }
+            // Compare inverted-list elements.
+            int count = 0;
+            actualInvListCursor.pinPages();
+            try {
+                while (actualInvListCursor.hasNext() && expectedInvListIter.hasNext()) {
+                    actualInvListCursor.next();
+                    ITupleReference actual = actualInvListCursor.getTuple();
+                    CheckTuple expected = expectedInvListIter.next();
+                    OrderedIndexTestUtils.createTupleFromCheckTuple(expected, expectedBuilder, completeExpectedTuple,
+                            fieldSerdes);
+                    expectedTuple.reset(completeExpectedTuple);
+                    if (invListCmp.compare(actual, expectedTuple) != 0) {
+                        fail("Inverted lists of token '" + token + "' differ at position " + count + ".");
+                    }
+                    count++;
+                }
+            } finally {
+                actualInvListCursor.unpinPages();
+            }
+        }
+    }
+
+    /**
+     * Determine the expected results with the simple ScanCount algorithm.
+     */
+    public static void getExpectedResults(int[] scanCountArray, TreeSet<CheckTuple> checkTuples,
+            ITupleReference searchDocument, IBinaryTokenizer tokenizer, ISerializerDeserializer tokenSerde,
+            IInvertedIndexSearchModifier searchModifier, List<Integer> expectedResults, InvertedIndexType invIndexType)
+            throws IOException {
+        boolean isPartitioned = false;
+        switch (invIndexType) {
+            case INMEMORY:
+            case ONDISK:
+            case LSM: {
+                isPartitioned = false;
+                break;
+            }
+            case PARTITIONED_INMEMORY:
+            case PARTITIONED_ONDISK:
+            case PARTITIONED_LSM: {
+                isPartitioned = true;
+                break;
+            }
+        }
+        getExpectedResults(scanCountArray, checkTuples, searchDocument, tokenizer, tokenSerde, searchModifier,
+                expectedResults, isPartitioned);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static void getExpectedResults(int[] scanCountArray, TreeSet<CheckTuple> checkTuples,
+            ITupleReference searchDocument, IBinaryTokenizer tokenizer, ISerializerDeserializer tokenSerde,
+            IInvertedIndexSearchModifier searchModifier, List<Integer> expectedResults, boolean isPartitioned)
+            throws IOException {
+        // Reset scan count array.
+        Arrays.fill(scanCountArray, 0);
+        expectedResults.clear();
+
+        GrowableArray tokenData = new GrowableArray();
+        tokenizer.reset(searchDocument.getFieldData(0), searchDocument.getFieldStart(0),
+                searchDocument.getFieldLength(0));
+        // Run though tokenizer to get number of tokens.
+        int numQueryTokens = 0;
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+            numQueryTokens++;
+        }
+        short numTokensLowerBound = -1;
+        short numTokensUpperBound = -1;
+        int invListElementField = 1;
+        if (isPartitioned) {
+            numTokensLowerBound = searchModifier.getNumTokensLowerBound((short) numQueryTokens);
+            numTokensUpperBound = searchModifier.getNumTokensUpperBound((short) numQueryTokens);
+            invListElementField = 2;
+        }
+        int occurrenceThreshold = searchModifier.getOccurrenceThreshold(numQueryTokens);
+        tokenizer.reset(searchDocument.getFieldData(0), searchDocument.getFieldStart(0),
+                searchDocument.getFieldLength(0));
+        while (tokenizer.hasNext()) {
+            tokenizer.next();
+            IToken token = tokenizer.getToken();
+            tokenData.reset();
+            token.serializeToken(tokenData);
+            ByteArrayInputStream inStream = new ByteArrayInputStream(tokenData.getByteArray(), 0, tokenData.getLength());
+            DataInput dataIn = new DataInputStream(inStream);
+            Comparable tokenObj = (Comparable) tokenSerde.deserialize(dataIn);
+            CheckTuple lowKey;
+            if (numTokensLowerBound < 0) {
+                // Index is not partitioned, or no length filtering is possible for this search modifier.
+                lowKey = new CheckTuple(1, 1);
+                lowKey.appendField(tokenObj);
+            } else {
+                // Index is length partitioned, and search modifier supports length filtering.
+                lowKey = new CheckTuple(2, 2);
+                lowKey.appendField(tokenObj);
+                lowKey.appendField(Short.valueOf(numTokensLowerBound));
+            }
+            CheckTuple highKey;
+            if (numTokensUpperBound < 0) {
+                // Index is not partitioned, or no length filtering is possible for this search modifier.
+                highKey = new CheckTuple(1, 1);
+                highKey.appendField(tokenObj);
+            } else {
+                // Index is length partitioned, and search modifier supports length filtering.
+                highKey = new CheckTuple(2, 2);
+                highKey.appendField(tokenObj);
+                highKey.appendField(Short.valueOf(numTokensUpperBound));
+            }
+
+            // Get view over check tuples containing inverted-list corresponding to token. 
+            SortedSet<CheckTuple> invList = OrderedIndexTestUtils.getPrefixExpectedSubset(checkTuples, lowKey, highKey);
+            Iterator<CheckTuple> invListIter = invList.iterator();
+            // Iterate over inverted list and update scan count array.
+            while (invListIter.hasNext()) {
+                CheckTuple checkTuple = invListIter.next();
+                Integer element = (Integer) checkTuple.getField(invListElementField);
+                scanCountArray[element]++;
+            }
+        }
+
+        // Run through scan count array, and see whether elements satisfy the given occurrence threshold.
+        expectedResults.clear();
+        for (int i = 0; i < scanCountArray.length; i++) {
+            if (scanCountArray[i] >= occurrenceThreshold) {
+                expectedResults.add(i);
+            }
+        }
+    }
+
+    public static void testIndexSearch(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen, Random rnd,
+            int numDocQueries, int numRandomQueries, IInvertedIndexSearchModifier searchModifier, int[] scanCountArray)
+            throws IOException, IndexException {
+        IInvertedIndex invIndex = testCtx.invIndex;
+        IInvertedIndexAccessor accessor = (IInvertedIndexAccessor) invIndex.createAccessor(
+                NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        IBinaryTokenizer tokenizer = testCtx.getTokenizerFactory().createTokenizer();
+        InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizer, searchModifier);
+        List<ITupleReference> documentCorpus = testCtx.getDocumentCorpus();
+        // Project away the primary-key field.
+        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.
+            if (i >= numDocQueries || i >= documentCorpus.size()) {
+                // Generate a random query.
+                ITupleReference randomQuery = tupleGen.next();
+                searchDocument.reset(randomQuery);
+            } else {
+                // Pick a random document from the corpus to use as the search query.
+                int queryIndex = Math.abs(rnd.nextInt() % documentCorpus.size());
+                searchDocument.reset(documentCorpus.get(queryIndex));
+            }
+
+            // Set query tuple in search predicate.
+            searchPred.setQueryTuple(searchDocument);
+            searchPred.setQueryFieldIndex(0);
+
+            resultCursor.reset();
+            boolean panic = false;
+            try {
+                accessor.search(resultCursor, searchPred);
+            } catch (OccurrenceThresholdPanicException e) {
+                // ignore panic queries.
+                panic = true;
+            }
+
+            try {
+                if (!panic) {
+                    // Consume cursor and deserialize results so we can sort them. Some search cursors may not deliver the result sorted (e.g., LSM search cursor).
+                    ArrayList<Integer> actualResults = new ArrayList<Integer>();
+                    try {
+                        while (resultCursor.hasNext()) {
+                            resultCursor.next();
+                            ITupleReference resultTuple = resultCursor.getTuple();
+                            int actual = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
+                                    resultTuple.getFieldStart(0));
+                            actualResults.add(Integer.valueOf(actual));
+                        }
+                    } catch (OccurrenceThresholdPanicException e) {
+                        // Ignore panic queries.
+                        continue;
+                    }
+                    Collections.sort(actualResults);
+
+                    // Get expected results.
+                    List<Integer> expectedResults = new ArrayList<Integer>();
+                    LSMInvertedIndexTestUtils.getExpectedResults(scanCountArray, testCtx.getCheckTuples(),
+                            searchDocument, tokenizer, testCtx.getFieldSerdes()[0], searchModifier, expectedResults,
+                            testCtx.getInvertedIndexType());
+
+                    Iterator<Integer> expectedIter = expectedResults.iterator();
+                    Iterator<Integer> actualIter = actualResults.iterator();
+                    while (expectedIter.hasNext() && actualIter.hasNext()) {
+                        int expected = expectedIter.next();
+                        int actual = actualIter.next();
+                        if (actual != expected) {
+                            fail("Query results do not match. Encountered: " + actual + ". Expected: " + expected + "");
+                        }
+                    }
+                    if (expectedIter.hasNext()) {
+                        fail("Query results do not match. Actual results missing.");
+                    }
+                    if (actualIter.hasNext()) {
+                        fail("Query results do not match. Actual contains too many results.");
+                    }
+                }
+            } finally {
+                resultCursor.close();
+            }
+        }
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
new file mode 100644
index 0000000..fe9267f
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
@@ -0,0 +1,40 @@
+<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>
+  <artifactId>hyracks-storage-am-lsm-rtree-test</artifactId>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks-tests</artifactId>
+    <version>0.2.3-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>  	
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-test-support</artifactId>
+  		<version>0.2.3-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  </dependencies>
+</project>
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
new file mode 100644
index 0000000..995f18c
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 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 LSMRTreeBulkLoadTest extends AbstractRTreeBulkLoadTest {
+
+    public LSMRTreeBulkLoadTest() {
+        super(1, 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.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @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/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
new file mode 100644
index 0000000..d72b668
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 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.AbstractRTreeDeleteTest;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+@SuppressWarnings("rawtypes")
+public class LSMRTreeDeleteTest extends AbstractRTreeDeleteTest {
+
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    public LSMRTreeDeleteTest() {
+        super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+    }
+
+    @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.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @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/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
new file mode 100644
index 0000000..cba8cee
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2010 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 org.junit.After;
+import org.junit.Before;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+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.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeExamplesTest;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class LSMRTreeExamplesTest extends AbstractRTreeExamplesTest {
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    @Override
+    protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType) throws TreeIndexException {
+        return LSMRTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(),
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+    }
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+}
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
new file mode 100644
index 0000000..96485f8
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 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.AbstractRTreeInsertTest;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+@SuppressWarnings("rawtypes")
+public class LSMRTreeInsertTest extends AbstractRTreeInsertTest {
+
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    public LSMRTreeInsertTest() {
+        super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+    }
+
+    @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.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @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/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
new file mode 100644
index 0000000..e72b3ca
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
@@ -0,0 +1,84 @@
+package edu.uci.ics.hyracks.storage.am.lsm.rtree;
+
+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;
+import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTree;
+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.RTreeTestUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
+
+public class LSMRTreeLifecycleTest extends AbstractIndexLifecycleTest {
+
+    @SuppressWarnings("rawtypes")
+    private final ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    private final IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils
+            .createPrimitiveValueProviderFactories(4, IntegerPointable.FACTORY);
+    private final int numKeys = 4;
+
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+    private final TreeIndexTestUtils titu = new RTreeTestUtils();
+
+    @SuppressWarnings("rawtypes")
+    private IIndexTestContext<? extends CheckTuple> testCtx;
+
+    @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;
+            }
+        }
+        return true;
+    }
+
+    @Override
+    protected boolean isEmptyIndex() throws Exception {
+        return ((LSMRTree) index).isEmptyIndex();
+    }
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        testCtx = LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+        index = testCtx.getIndex();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        index.deactivate();
+        index.destroy();
+        harness.tearDown();
+    }
+
+    @Override
+    protected void performInsertions() throws Exception {
+        titu.insertIntTuples(testCtx, 10, harness.getRandom());
+    }
+
+    @Override
+    protected void checkInsertions() throws Exception {
+        titu.checkScan(testCtx);
+    }
+
+    @Override
+    protected void clearCheckableInsertions() throws Exception {
+        testCtx.getCheckTuples().clear();
+    }
+}
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
new file mode 100644
index 0000000..1d07484
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2010 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.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+@SuppressWarnings("rawtypes")
+public class LSMRTreeMergeTest extends LSMRTreeMergeTestDriver {
+
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    public LSMRTreeMergeTest() {
+        super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+    }
+
+    @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.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @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/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
new file mode 100644
index 0000000..f272942
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2010 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 edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+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.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestDriver;
+import edu.uci.ics.hyracks.storage.am.rtree.RTreeTestUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+@SuppressWarnings("rawtypes")
+public abstract class LSMRTreeMergeTestDriver extends AbstractRTreeTestDriver {
+
+    private final RTreeTestUtils rTreeTestUtils;
+
+    public LSMRTreeMergeTestDriver(boolean testRstarPolicy) {
+        super(testRstarPolicy);
+        this.rTreeTestUtils = new RTreeTestUtils();
+    }
+
+    @Override
+    protected void runTest(ISerializerDeserializer[] fieldSerdes,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key,
+            RTreePolicyType rtreePolicyType) throws Exception {
+
+        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
+        // Start off with one tree bulk loaded.
+        // 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());
+        }
+
+        int maxTreesToMerge = AccessMethodTestsConfig.LSM_RTREE_BULKLOAD_ROUNDS;
+        for (int i = 0; i < maxTreesToMerge; i++) {
+            for (int j = 0; j < i; j++) {
+                if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
+                    rTreeTestUtils.bulkLoadIntTuples(ctx, numTuplesToInsert, getRandom());
+                } else if (fieldSerdes[0] instanceof DoubleSerializerDeserializer) {
+                    rTreeTestUtils.bulkLoadDoubleTuples(ctx, numTuplesToInsert, getRandom());
+                }
+            }
+
+            ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
+            accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+
+            rTreeTestUtils.checkScan(ctx);
+            rTreeTestUtils.checkDiskOrderScan(ctx);
+            rTreeTestUtils.checkRangeSearch(ctx, key);
+        }
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Override
+    protected String getTestOpName() {
+        return "LSM Merge";
+    }
+}
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
new file mode 100644
index 0000000..cc46065
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2010 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.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider());
+    }
+
+    @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
new file mode 100644
index 0000000..81a952d
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 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 LSMRTreeWithAntiMatterTuplesBulkLoadTest extends AbstractRTreeBulkLoadTest {
+
+    public LSMRTreeWithAntiMatterTuplesBulkLoadTest() {
+        super(1, 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.getMemBufferCache(),
+                harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+
+    }
+
+    @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/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
new file mode 100644
index 0000000..1ee92d9
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 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.AbstractRTreeDeleteTest;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+@SuppressWarnings("rawtypes")
+public class LSMRTreeWithAntiMatterTuplesDeleteTest extends AbstractRTreeDeleteTest {
+
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    public LSMRTreeWithAntiMatterTuplesDeleteTest() {
+        super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+    }
+
+    @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.getMemBufferCache(),
+                harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+
+    }
+
+    @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/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
new file mode 100644
index 0000000..3a2537c
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2010 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 org.junit.After;
+import org.junit.Before;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+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.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeExamplesTest;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class LSMRTreeWithAntiMatterTuplesExamplesTest extends AbstractRTreeExamplesTest {
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    @Override
+    protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType) throws TreeIndexException {
+        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getMemBufferCache(),
+                harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
+                btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(),
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+    }
+
+    @Before
+    public void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+}
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
new file mode 100644
index 0000000..61d5ce7
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 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.AbstractRTreeInsertTest;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+@SuppressWarnings("rawtypes")
+public class LSMRTreeWithAntiMatterTuplesInsertTest extends AbstractRTreeInsertTest {
+
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    public LSMRTreeWithAntiMatterTuplesInsertTest() {
+        super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+    }
+
+    @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.getMemBufferCache(),
+                harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+
+    }
+
+    @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/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
new file mode 100644
index 0000000..aee8670
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
@@ -0,0 +1,84 @@
+package edu.uci.ics.hyracks.storage.am.lsm.rtree;
+
+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;
+import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuples;
+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.RTreeTestUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
+
+public class LSMRTreeWithAntiMatterTuplesLifecycleTest extends AbstractIndexLifecycleTest {
+
+    @SuppressWarnings("rawtypes")
+    private final ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    private final IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils
+            .createPrimitiveValueProviderFactories(4, IntegerPointable.FACTORY);
+    private final int numKeys = 4;
+
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+    private final TreeIndexTestUtils titu = new RTreeTestUtils();
+
+    @SuppressWarnings("rawtypes")
+    private IIndexTestContext<? extends CheckTuple> testCtx;
+
+    @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;
+            }
+        }
+        return true;
+    }
+
+    @Override
+    protected boolean isEmptyIndex() throws Exception {
+        return ((LSMRTreeWithAntiMatterTuples) index).isEmptyIndex();
+    }
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getMemBufferCache(),
+                harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+        index = testCtx.getIndex();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        index.deactivate();
+        index.destroy();
+        harness.tearDown();
+    }
+
+    @Override
+    protected void performInsertions() throws Exception {
+        titu.insertIntTuples(testCtx, 10, harness.getRandom());
+    }
+
+    @Override
+    protected void checkInsertions() throws Exception {
+        titu.checkScan(testCtx);
+    }
+
+    @Override
+    protected void clearCheckableInsertions() throws Exception {
+        testCtx.getCheckTuples().clear();
+    }
+}
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
new file mode 100644
index 0000000..d5fecbf
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2010 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.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+@SuppressWarnings("rawtypes")
+public class LSMRTreeWithAntiMatterTuplesMergeTest extends LSMRTreeMergeTestDriver {
+
+    private final LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    public LSMRTreeWithAntiMatterTuplesMergeTest() {
+        super(AccessMethodTestsConfig.LSM_RTREE_TEST_RSTAR_POLICY);
+    }
+
+    @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.getMemBufferCache(),
+                harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+    }
+
+    @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/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
new file mode 100644
index 0000000..de5f065
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 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.getMemBufferCache(),
+                harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
+                numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+
+    }
+
+    @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
new file mode 100644
index 0000000..d530b82
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/AbstractLSMRTreeTestWorker.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2010 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.multithread;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+
+public abstract class AbstractLSMRTreeTestWorker extends AbstractIndexTestWorker {
+
+    protected final ITreeIndex lsmRTree;
+    protected final int numFields;
+    protected final ArrayTupleBuilder rearrangedTb;
+    protected final ArrayTupleReference rearrangedTuple = new ArrayTupleReference();
+
+    public AbstractLSMRTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+            int numBatches) {
+        super(dataGen, opSelector, index, numBatches);
+        lsmRTree = (ITreeIndex) index;
+        numFields = lsmRTree.getFieldCount();
+        rearrangedTb = new ArrayTupleBuilder(numFields);
+    }
+
+    protected void rearrangeTuple(ITupleReference tuple, MultiComparator cmp) throws HyracksDataException {
+        // Create a tuple with rearranged key values to make sure lower points
+        // have larger coordinates than high points.
+        rearrangedTb.reset();
+        int maxFieldPos = cmp.getKeyFieldCount() / 2;
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            int c = cmp.getComparators()[i].compare(tuple.getFieldData(i), tuple.getFieldStart(i),
+                    tuple.getFieldLength(i), tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j));
+            if (c > 0) {
+                rearrangedTb.addField(tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j));
+            } else {
+                rearrangedTb.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+            }
+        }
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            int c = cmp.getComparators()[i].compare(tuple.getFieldData(i), tuple.getFieldStart(i),
+                    tuple.getFieldLength(i), tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j));
+            if (c > 0) {
+                rearrangedTb.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+            } else {
+                rearrangedTb.addField(tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j));
+            }
+        }
+        for (int i = cmp.getKeyFieldCount(); i < numFields; i++) {
+            rearrangedTb.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+        }
+        rearrangedTuple.reset(rearrangedTb.getFieldEndOffsets(), rearrangedTb.getByteArray());
+    }
+
+    protected void consumeCursorTuples(ITreeIndexCursor cursor) throws HyracksDataException, IndexException {
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+            }
+        } finally {
+            cursor.close();
+        }
+    }
+}
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
new file mode 100644
index 0000000..37be58d
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2009-2010 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.multithread;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.ProbabilityHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeMultiThreadTest;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class LSMRTreeMultiThreadTest extends AbstractRTreeMultiThreadTest {
+
+    private LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    private LSMRTreeTestWorkerFactory workerFactory = new LSMRTreeTestWorkerFactory();
+
+    public LSMRTreeMultiThreadTest() {
+        super(false);
+    }
+
+    @Override
+    protected void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @Override
+    protected void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType) throws TreeIndexException {
+        return LSMRTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
+                harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
+                harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(),
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+    }
+
+    @Override
+    protected IIndexTestWorkerFactory getWorkerFactory() {
+        return workerFactory;
+    }
+
+    @Override
+    protected ArrayList<TestWorkloadConf> getTestWorkloadConf() {
+        ArrayList<TestWorkloadConf> workloadConfs = new ArrayList<TestWorkloadConf>();
+
+        // Insert only workload.
+        TestOperation[] insertOnlyOps = new TestOperation[] { TestOperation.INSERT };
+        workloadConfs.add(new TestWorkloadConf(insertOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertOnlyOps.length)));
+
+        // Insert and merge workload.
+        TestOperation[] insertMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(insertMergeOps, ProbabilityHelper
+                .getUniformProbDist(insertMergeOps.length)));
+
+        // Inserts mixed with scans.
+        TestOperation[] insertSearchOnlyOps = new TestOperation[] { TestOperation.INSERT, TestOperation.SCAN };
+        workloadConfs.add(new TestWorkloadConf(insertSearchOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertSearchOnlyOps.length)));
+
+        // Inserts and deletes.
+        TestOperation[] insertDeleteOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE };
+        workloadConfs.add(new TestWorkloadConf(insertDeleteOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteOps.length)));
+
+        // Inserts, deletes and merges.
+        TestOperation[] insertDeleteMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(insertDeleteMergeOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteMergeOps.length)));
+
+        // All operations except merge.
+        TestOperation[] allNoMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.SCAN };
+        workloadConfs.add(new TestWorkloadConf(allNoMergeOps, ProbabilityHelper
+                .getUniformProbDist(allNoMergeOps.length)));
+
+        // All operations.
+        TestOperation[] allOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE, TestOperation.SCAN,
+                TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(allOps, ProbabilityHelper.getUniformProbDist(allOps.length)));
+
+        return workloadConfs;
+    }
+
+    @Override
+    protected String getIndexTypeName() {
+        return "LSMRTree";
+    }
+
+}
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
new file mode 100644
index 0000000..154756e
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2009-2010 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.multithread;
+
+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.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTree;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTree.LSMRTreeAccessor;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
+
+public class LSMRTreeTestWorker extends AbstractIndexTestWorker {
+
+    private final LSMRTree lsmRTree;
+    private final int numFields;
+    private final ArrayTupleBuilder rearrangedTb;
+    private final ArrayTupleReference rearrangedTuple = new ArrayTupleReference();
+
+    public LSMRTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
+        super(dataGen, opSelector, index, numBatches);
+        lsmRTree = (LSMRTree) index;
+        numFields = lsmRTree.getFieldCount();
+        rearrangedTb = new ArrayTupleBuilder(numFields);
+    }
+
+    @Override
+    public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
+        LSMRTreeAccessor accessor = (LSMRTreeAccessor) indexAccessor;
+        ITreeIndexCursor searchCursor = accessor.createSearchCursor();
+        MultiComparator cmp = accessor.getMultiComparator();
+        SearchPredicate rangePred = new SearchPredicate(tuple, cmp);
+
+        switch (op) {
+            case INSERT:
+                rearrangeTuple(tuple, cmp);
+                accessor.insert(rearrangedTuple);
+                break;
+
+            case DELETE:
+                rearrangeTuple(tuple, cmp);
+                accessor.delete(rearrangedTuple);
+                break;
+
+            case SCAN:
+                searchCursor.reset();
+                rangePred.setSearchKey(null);
+                accessor.search(searchCursor, rangePred);
+                consumeCursorTuples(searchCursor);
+                break;
+
+            case MERGE:
+                accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+                break;
+
+            default:
+                throw new HyracksDataException("Op " + op.toString() + " not supported.");
+        }
+    }
+
+    private void rearrangeTuple(ITupleReference tuple, MultiComparator cmp) throws HyracksDataException {
+        // Create a tuple with rearranged key values to make sure lower points
+        // have larger coordinates than high points.
+        rearrangedTb.reset();
+        int maxFieldPos = cmp.getKeyFieldCount() / 2;
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            int c = cmp.getComparators()[i].compare(tuple.getFieldData(i), tuple.getFieldStart(i),
+                    tuple.getFieldLength(i), tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j));
+            if (c > 0) {
+                rearrangedTb.addField(tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j));
+            } else {
+                rearrangedTb.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+            }
+        }
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            int c = cmp.getComparators()[i].compare(tuple.getFieldData(i), tuple.getFieldStart(i),
+                    tuple.getFieldLength(i), tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j));
+            if (c > 0) {
+                rearrangedTb.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+            } else {
+                rearrangedTb.addField(tuple.getFieldData(j), tuple.getFieldStart(j), tuple.getFieldLength(j));
+            }
+        }
+        for (int i = cmp.getKeyFieldCount(); i < numFields; i++) {
+            rearrangedTb.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+        }
+        rearrangedTuple.reset(rearrangedTb.getFieldEndOffsets(), rearrangedTb.getByteArray());
+    }
+
+    private void consumeCursorTuples(ITreeIndexCursor cursor) throws HyracksDataException, IndexException {
+        try {
+            while (cursor.hasNext()) {
+                cursor.next();
+            }
+        } finally {
+            cursor.close();
+        }
+    }
+}
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
new file mode 100644
index 0000000..13e3ab0
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorkerFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 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.multithread;
+
+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;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+
+public class LSMRTreeTestWorkerFactory implements IIndexTestWorkerFactory {
+    @Override
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
+            IIndex index, int numBatches) {
+        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
new file mode 100644
index 0000000..af73676
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright 2009-2010 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.multithread;
+
+import java.util.ArrayList;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.ProbabilityHelper;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeMultiThreadTest;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+
+public class LSMRTreeWithAntiMatterTuplesMultiThreadTest extends AbstractRTreeMultiThreadTest {
+
+    private LSMRTreeTestHarness harness = new LSMRTreeTestHarness();
+
+    private LSMRTreeWithAntiMatterTuplesTestWorkerFactory workerFactory = new LSMRTreeWithAntiMatterTuplesTestWorkerFactory();
+
+    public LSMRTreeWithAntiMatterTuplesMultiThreadTest() {
+        super(false);
+    }
+
+    @Override
+    protected void setUp() throws HyracksException {
+        harness.setUp();
+    }
+
+    @Override
+    protected void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @Override
+    protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType) throws TreeIndexException {
+        return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(harness.getMemBufferCache(),
+                harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
+                harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
+                btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(),
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+
+    }
+
+    @Override
+    protected IIndexTestWorkerFactory getWorkerFactory() {
+        return workerFactory;
+    }
+
+    @Override
+    protected ArrayList<TestWorkloadConf> getTestWorkloadConf() {
+        ArrayList<TestWorkloadConf> workloadConfs = new ArrayList<TestWorkloadConf>();
+
+        // Insert only workload.
+        TestOperation[] insertOnlyOps = new TestOperation[] { TestOperation.INSERT };
+        workloadConfs.add(new TestWorkloadConf(insertOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertOnlyOps.length)));
+
+        // Insert and merge workload.
+        TestOperation[] insertMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(insertMergeOps, ProbabilityHelper
+                .getUniformProbDist(insertMergeOps.length)));
+
+        // Inserts mixed with scans.
+        TestOperation[] insertSearchOnlyOps = new TestOperation[] { TestOperation.INSERT, TestOperation.SCAN };
+        workloadConfs.add(new TestWorkloadConf(insertSearchOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertSearchOnlyOps.length)));
+
+        // Inserts and deletes.
+        TestOperation[] insertDeleteOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE };
+        workloadConfs.add(new TestWorkloadConf(insertDeleteOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteOps.length)));
+
+        // Inserts, deletes and merges.
+        TestOperation[] insertDeleteMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(insertDeleteMergeOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteMergeOps.length)));
+
+        // All operations except merge.
+        TestOperation[] allNoMergeOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE,
+                TestOperation.SCAN };
+        workloadConfs.add(new TestWorkloadConf(allNoMergeOps, ProbabilityHelper
+                .getUniformProbDist(allNoMergeOps.length)));
+
+        // All operations.
+        TestOperation[] allOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE, TestOperation.SCAN,
+                TestOperation.MERGE };
+        workloadConfs.add(new TestWorkloadConf(allOps, ProbabilityHelper.getUniformProbDist(allOps.length)));
+
+        return workloadConfs;
+    }
+
+    @Override
+    protected String getIndexTypeName() {
+        return "LSMRTree";
+    }
+
+}
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
new file mode 100644
index 0000000..661138d
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2010 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.multithread;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuples.LSMRTreeWithAntiMatterTuplesAccessor;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
+
+public class LSMRTreeWithAntiMatterTuplesTestWorker extends AbstractLSMRTreeTestWorker {
+
+    public LSMRTreeWithAntiMatterTuplesTestWorker(DataGenThread dataGen, TestOperationSelector opSelector,
+            IIndex index, int numBatches) {
+        super(dataGen, opSelector, index, numBatches);
+    }
+
+    @Override
+    public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
+        LSMRTreeWithAntiMatterTuplesAccessor accessor = (LSMRTreeWithAntiMatterTuplesAccessor) indexAccessor;
+        ITreeIndexCursor searchCursor = accessor.createSearchCursor();
+        MultiComparator cmp = accessor.getMultiComparator();
+        SearchPredicate rangePred = new SearchPredicate(tuple, cmp);
+
+        switch (op) {
+            case INSERT:
+                rearrangeTuple(tuple, cmp);
+                accessor.insert(rearrangedTuple);
+                break;
+
+            case DELETE:
+                rearrangeTuple(tuple, cmp);
+                accessor.delete(rearrangedTuple);
+                break;
+
+            case SCAN:
+                searchCursor.reset();
+                rangePred.setSearchKey(null);
+                accessor.search(searchCursor, rangePred);
+                consumeCursorTuples(searchCursor);
+                break;
+
+            case MERGE:
+                accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+                break;
+
+            default:
+                throw new HyracksDataException("Op " + op.toString() + " not supported.");
+        }
+    }
+}
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
new file mode 100644
index 0000000..4e78d82
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorkerFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 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.multithread;
+
+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;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+
+public class LSMRTreeWithAntiMatterTuplesTestWorkerFactory implements IIndexTestWorkerFactory {
+    @Override
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
+            IIndex index, int numBatches) {
+        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
new file mode 100644
index 0000000..27fb9c8
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2010 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.util;
+
+import java.util.Collection;
+
+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.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.IInMemoryFreePageManager;
+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.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTree;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.RTreeCheckTuple;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+@SuppressWarnings("rawtypes")
+public final class LSMRTreeTestContext extends AbstractRTreeTestContext {
+
+    public LSMRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+        super(fieldSerdes, treeIndex);
+    }
+
+    @Override
+    public int getKeyFieldCount() {
+        LSMRTree lsmTree = (LSMRTree) index;
+        return lsmTree.getComparatorFactories().length;
+    }
+
+    /**
+     * Override to provide delete semantics for the check tuples.
+     */
+    @Override
+    public void deleteCheckTuple(RTreeCheckTuple checkTuple, Collection<RTreeCheckTuple> checkTuples) {
+        while (checkTuples.remove(checkTuple)) {
+        }
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getComparatorFactories() {
+        LSMRTree lsmTree = (LSMRTree) index;
+        return lsmTree.getComparatorFactories();
+    }
+
+    public static LSMRTreeTestContext create(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            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(memBufferCache, memFreePageManager, ioManager, file,
+                diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, 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
new file mode 100644
index 0000000..6fb6f9c
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -0,0 +1,214 @@
+/*
+ * Copyright 2009-2010 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.util;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Random;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public class LSMRTreeTestHarness {
+    protected static final Logger LOGGER = Logger.getLogger(LSMRTreeTestHarness.class.getName());
+
+    private static final long RANDOM_SEED = 50;
+
+    protected final int diskPageSize;
+    protected final int diskNumPages;
+    protected final int diskMaxOpenFiles;
+    protected final int memPageSize;
+    protected final int memNumPages;
+    protected final int hyracksFrameSize;
+
+    protected IOManager ioManager;
+    protected IBufferCache diskBufferCache;
+    protected IFileMapProvider diskFileMapProvider;
+    protected IInMemoryBufferCache memBufferCache;
+    protected IInMemoryFreePageManager memFreePageManager;
+    protected IHyracksTaskContext ctx;
+    protected ILSMIOOperationScheduler ioScheduler;
+    protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    protected ILSMMergePolicy mergePolicy;
+    protected ILSMOperationTrackerFactory opTrackerFactory;
+
+    protected final Random rnd = new Random();
+    protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    protected final static String sep = System.getProperty("file.separator");
+    protected String onDiskDir;
+    protected FileReference file;
+
+    public LSMRTreeTestHarness() {
+        this.diskPageSize = AccessMethodTestsConfig.LSM_RTREE_DISK_PAGE_SIZE;
+        this.diskNumPages = AccessMethodTestsConfig.LSM_RTREE_DISK_NUM_PAGES;
+        this.diskMaxOpenFiles = AccessMethodTestsConfig.LSM_RTREE_DISK_MAX_OPEN_FILES;
+        this.memPageSize = AccessMethodTestsConfig.LSM_RTREE_MEM_PAGE_SIZE;
+        this.memNumPages = AccessMethodTestsConfig.LSM_RTREE_MEM_NUM_PAGES;
+        this.hyracksFrameSize = AccessMethodTestsConfig.LSM_RTREE_HYRACKS_FRAME_SIZE;
+        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.mergePolicy = NoMergePolicy.INSTANCE;
+        this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+        this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
+    }
+
+    public LSMRTreeTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
+            int memNumPages, int hyracksFrameSize) {
+        this.diskPageSize = diskPageSize;
+        this.diskNumPages = diskNumPages;
+        this.diskMaxOpenFiles = diskMaxOpenFiles;
+        this.memPageSize = memPageSize;
+        this.memNumPages = memNumPages;
+        this.hyracksFrameSize = hyracksFrameSize;
+        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.mergePolicy = NoMergePolicy.INSTANCE;
+        this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
+    }
+
+    public void setUp() throws HyracksException {
+        onDiskDir = "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);
+        memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
+        memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages, new LIFOMetaDataFrameFactory());
+        ioManager = TestStorageManagerComponentHolder.getIOManager();
+        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();
+                }
+            }
+            dir.delete();
+        }
+    }
+
+    public int getDiskPageSize() {
+        return diskPageSize;
+    }
+
+    public int getDiskNumPages() {
+        return diskNumPages;
+    }
+
+    public int getDiskMaxOpenFiles() {
+        return diskMaxOpenFiles;
+    }
+
+    public int getMemPageSize() {
+        return memPageSize;
+    }
+
+    public int getMemNumPages() {
+        return memNumPages;
+    }
+
+    public int getHyracksFrameSize() {
+        return hyracksFrameSize;
+    }
+
+    public IOManager getIOManager() {
+        return ioManager;
+    }
+
+    public IBufferCache getDiskBufferCache() {
+        return diskBufferCache;
+    }
+
+    public IFileMapProvider getDiskFileMapProvider() {
+        return diskFileMapProvider;
+    }
+
+    public IInMemoryBufferCache getMemBufferCache() {
+        return memBufferCache;
+    }
+
+    public IInMemoryFreePageManager getMemFreePageManager() {
+        return memFreePageManager;
+    }
+
+    public IHyracksTaskContext getHyracksTastContext() {
+        return ctx;
+    }
+
+    public String getOnDiskDir() {
+        return onDiskDir;
+    }
+
+    public FileReference getFileReference() {
+        return file;
+    }
+
+    public Random getRandom() {
+        return rnd;
+    }
+
+    public ILSMIOOperationScheduler getIOScheduler() {
+        return ioScheduler;
+    }
+
+    public ILSMOperationTrackerFactory getOperationTrackerFactory() {
+        return opTrackerFactory;
+    }
+
+    public ILSMMergePolicy getMergePolicy() {
+        return mergePolicy;
+    }
+
+    public ILSMIOOperationCallbackProvider getIOOperationCallbackProvider() {
+        return ioOpCallbackProvider;
+    }
+}
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
new file mode 100644
index 0000000..5860236
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2009-2010 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.util;
+
+import java.util.Collection;
+
+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.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.IInMemoryFreePageManager;
+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.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeWithAntiMatterTuples;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.RTreeCheckTuple;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+@SuppressWarnings("rawtypes")
+public final class LSMRTreeWithAntiMatterTuplesTestContext extends AbstractRTreeTestContext {
+
+    public LSMRTreeWithAntiMatterTuplesTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+        super(fieldSerdes, treeIndex);
+    }
+
+    @Override
+    public int getKeyFieldCount() {
+        LSMRTreeWithAntiMatterTuples lsmTree = (LSMRTreeWithAntiMatterTuples) index;
+        return lsmTree.getComparatorFactories().length;
+    }
+
+    /**
+     * Override to provide delete semantics for the check tuples.
+     */
+    @Override
+    public void deleteCheckTuple(RTreeCheckTuple checkTuple, Collection<RTreeCheckTuple> checkTuples) {
+        while (checkTuples.remove(checkTuple)) {
+        }
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getComparatorFactories() {
+        LSMRTreeWithAntiMatterTuples lsmTree = (LSMRTreeWithAntiMatterTuples) index;
+        return lsmTree.getComparatorFactories();
+    }
+
+    public static LSMRTreeWithAntiMatterTuplesTestContext create(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IOManager ioManager, FileReference file,
+            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            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(memBufferCache,
+                memFreePageManager, ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits,
+                rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, mergePolicy,
+                opTrackerFactory, 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/pom.xml b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
index 7b1a3f3..6738b08 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
@@ -1,8 +1,6 @@
 <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>hyracks-storage-am-rtree-test</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-storage-am-rtree-test</name>
 
   <parent>
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 58bca10..4eeec1a 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
@@ -23,8 +23,8 @@
 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.api.IPrimitiveValueProviderFactory;
-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.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestContext;
 import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestHarness;
 
@@ -32,7 +32,7 @@
 public class RTreeBulkLoadTest extends AbstractRTreeBulkLoadTest {
 
     public RTreeBulkLoadTest() {
-        super(1);
+        super(1, AccessMethodTestsConfig.RTREE_TEST_RSTAR_POLICY);
     }
 
     private final RTreeTestHarness harness = new RTreeTestHarness();
@@ -49,9 +49,10 @@
 
     @Override
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys) throws Exception {
-        return RTreeTestContext.create(harness.getBufferCache(), harness.getTreeFileId(), fieldSerdes,
-                valueProviderFactories, numKeys);
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
+            throws Exception {
+        return RTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeDeleteTest.java
index 42e933e..f4e45a8 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeDeleteTest.java
@@ -23,16 +23,20 @@
 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.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeDeleteTest;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestContext;
 import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestHarness;
 
 @SuppressWarnings("rawtypes")
 public class RTreeDeleteTest extends AbstractRTreeDeleteTest {
 
-    private final RTreeTestHarness harness = new RTreeTestHarness();
+	private final RTreeTestHarness harness = new RTreeTestHarness();
 
+	public RTreeDeleteTest() {
+		super(AccessMethodTestsConfig.RTREE_TEST_RSTAR_POLICY);
+	}
+	
     @Before
     public void setUp() throws HyracksDataException {
         harness.setUp();
@@ -45,9 +49,10 @@
 
     @Override
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys) throws Exception {
-        return RTreeTestContext.create(harness.getBufferCache(), harness.getTreeFileId(), fieldSerdes,
-                valueProviderFactories, numKeys);
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
+            throws Exception {
+        return RTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeExamplesTest.java
index c72338e..84c8642 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeExamplesTest.java
@@ -24,7 +24,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeExamplesTest;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestHarness;
 
@@ -43,14 +43,10 @@
 
     @Override
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories)
-            throws TreeIndexException {
-        return RTreeUtils.createRTree(harness.getBufferCache(), typeTraits,
-                valueProviderFactories, rtreeCmpFactories);
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType) throws TreeIndexException {
+        return RTreeUtils.createRTree(harness.getBufferCache(), harness.getFileMapProvider(), typeTraits,
+                valueProviderFactories, rtreeCmpFactories, rtreePolicyType, harness.getFileReference());
     }
 
-    @Override
-    protected int getIndexFileId() {
-        return harness.getTreeFileId();
-    }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeInsertTest.java
index 6efa620..065f687 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeInsertTest.java
@@ -23,26 +23,20 @@
 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.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeInsertTest;
-import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestContext;
 import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestHarness;
 
-/**
- * Tests the BTree insert operation with strings and integer fields using
- * various numbers of key and payload fields.
- * 
- * Each tests first fills a BTree with randomly generated tuples. We compare the
- * following operations against expected results: 1. Point searches for all
- * tuples. 2. Ordered scan. 3. Disk-order scan. 4. Range search (and prefix
- * search for composite keys).
- * 
- */
 @SuppressWarnings("rawtypes")
 public class RTreeInsertTest extends AbstractRTreeInsertTest {
 
-    private final RTreeTestHarness harness = new RTreeTestHarness();
+	private final RTreeTestHarness harness = new RTreeTestHarness();
 
+	public RTreeInsertTest() {
+		super(AccessMethodTestsConfig.RTREE_TEST_RSTAR_POLICY);
+	}
+	
     @Before
     public void setUp() throws HyracksDataException {
         harness.setUp();
@@ -55,9 +49,10 @@
 
     @Override
     protected AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys) throws Exception {
-        return RTreeTestContext.create(harness.getBufferCache(), harness.getTreeFileId(), fieldSerdes,
-                valueProviderFactories, numKeys);
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
+            throws Exception {
+        return RTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeLifecycleTest.java
new file mode 100644
index 0000000..69c7e15
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeLifecycleTest.java
@@ -0,0 +1,79 @@
+package edu.uci.ics.hyracks.storage.am.rtree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+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.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.common.AbstractIndexLifecycleTest;
+import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestHarness;
+
+public class RTreeLifecycleTest extends AbstractIndexLifecycleTest {
+    private final RTreeTestHarness harness = new RTreeTestHarness();
+    private final TreeIndexTestUtils titu = new RTreeTestUtils();
+
+    @SuppressWarnings("rawtypes")
+    private final ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    private final IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils
+            .createPrimitiveValueProviderFactories(4, IntegerPointable.FACTORY);
+    private final int numKeys = 4;
+
+    @SuppressWarnings("rawtypes")
+    private IIndexTestContext<? extends CheckTuple> testCtx;
+    private ITreeIndexFrame frame = null;
+
+    @Override
+    public void setup() throws Exception {
+        harness.setUp();
+        testCtx = RTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE);
+        index = testCtx.getIndex();
+    }
+
+    @Override
+    public void tearDown() throws HyracksDataException {
+        testCtx.getIndex().deactivate();
+        testCtx.getIndex().destroy();
+        harness.tearDown();
+    }
+
+    @Override
+    protected boolean persistentStateExists() {
+        return harness.getFileReference().getFile().exists();
+    }
+
+    @Override
+    protected boolean isEmptyIndex() throws HyracksDataException {
+        RTree rtree = (RTree) testCtx.getIndex();
+        if (frame == null) {
+            frame = rtree.getInteriorFrameFactory().createFrame();
+        }
+        return rtree.isEmptyTree(frame);
+    }
+
+    @Override
+    protected void performInsertions() throws Exception {
+        titu.insertIntTuples(testCtx, 10, harness.getRandom());
+    }
+
+    @Override
+    protected void checkInsertions() throws Exception {
+        titu.checkScan(testCtx);
+    }
+
+    @Override
+    protected void clearCheckableInsertions() throws Exception {
+        testCtx.getCheckTuples().clear();
+    }
+
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeSearchCursorTest.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
index 8332c11..06d18bb 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
@@ -41,13 +41,14 @@
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
+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.rtree.RTreeCheckTuple;
-import edu.uci.ics.hyracks.storage.am.rtree.RTreeTestUtils;
+import edu.uci.ics.hyracks.storage.am.common.util.HashMultiSet;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
@@ -78,7 +79,6 @@
         }
 
         IBufferCache bufferCache = harness.getBufferCache();
-        int rtreeFileId = harness.getTreeFileId();
 
         // Declare fields.
         int fieldCount = 5;
@@ -109,22 +109,23 @@
         ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
         ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
-                valueProviderFactories);
+                valueProviderFactories, RTreePolicyType.RTREE);
         ITreeIndexFrameFactory leafFrameFactory = new RTreeNSMLeafFrameFactory(tupleWriterFactory,
-                valueProviderFactories);
+                valueProviderFactories, RTreePolicyType.RTREE);
 
         IRTreeInteriorFrame interiorFrame = (IRTreeInteriorFrame) interiorFrameFactory.createFrame();
         IRTreeLeafFrame leafFrame = (IRTreeLeafFrame) leafFrameFactory.createFrame();
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaFrameFactory);
 
-        RTree rtree = new RTree(bufferCache, fieldCount, cmpFactories, freePageManager, interiorFrameFactory,
-                leafFrameFactory);
-        rtree.create(rtreeFileId);
-        rtree.open(rtreeFileId);
+        RTree rtree = new RTree(bufferCache, harness.getFileMapProvider(), freePageManager, interiorFrameFactory,
+                leafFrameFactory, cmpFactories, fieldCount, harness.getFileReference());
+        rtree.create();
+        rtree.activate();
 
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        ITreeIndexAccessor indexAccessor = rtree.createAccessor();
+        ITreeIndexAccessor indexAccessor = rtree.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
         int numInserts = 10000;
         ArrayList<RTreeCheckTuple> checkTuples = new ArrayList<RTreeCheckTuple>();
         for (int i = 0; i < numInserts; i++) {
@@ -142,11 +143,11 @@
             } catch (TreeIndexException e) {
             }
             RTreeCheckTuple checkTuple = new RTreeCheckTuple(fieldCount, keyFieldCount);
-            checkTuple.add(Math.min(p1x, p2x));
-            checkTuple.add(Math.min(p1y, p2y));
-            checkTuple.add(Math.max(p1x, p2x));
-            checkTuple.add(Math.max(p1y, p2y));
-            checkTuple.add(pk);
+            checkTuple.appendField(Math.min(p1x, p2x));
+            checkTuple.appendField(Math.min(p1y, p2y));
+            checkTuple.appendField(Math.max(p1x, p2x));
+            checkTuple.appendField(Math.max(p1y, p2y));
+            checkTuple.appendField(pk);
 
             checkTuples.add(checkTuple);
         }
@@ -162,14 +163,16 @@
 
         RTreeCheckTuple keyCheck = (RTreeCheckTuple) rTreeTestUtils.createCheckTupleFromTuple(key, fieldSerdes,
                 keyFieldCount);
-        ArrayList<RTreeCheckTuple> expectedResult = rTreeTestUtils.getRangeSearchExpectedResults(checkTuples, keyCheck);
+        HashMultiSet<RTreeCheckTuple> expectedResult = rTreeTestUtils.getRangeSearchExpectedResults(checkTuples,
+                keyCheck);
 
         rTreeTestUtils.getRangeSearchExpectedResults(checkTuples, keyCheck);
         indexAccessor.search(searchCursor, searchPredicate);
 
         rTreeTestUtils.checkExpectedResults(searchCursor, expectedResult, fieldSerdes, keyFieldCount, null);
 
-        rtree.close();
+        rtree.deactivate();
+        rtree.destroy();
     }
 
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
index 7520793..ee245e8 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeMultiThreadTest.java
@@ -20,18 +20,24 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestWorkerFactory;
-import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.ProbabilityHelper;
 import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeMultiThreadTest;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 import edu.uci.ics.hyracks.storage.am.rtree.utils.RTreeTestHarness;
 
 public class RTreeMultiThreadTest extends AbstractRTreeMultiThreadTest {
 
+    public RTreeMultiThreadTest() {
+        super(true);
+    }
+
     private RTreeTestHarness harness = new RTreeTestHarness();
 
     private RTreeTestWorkerFactory workerFactory = new RTreeTestWorkerFactory();
@@ -48,15 +54,15 @@
 
     @Override
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories)
-            throws TreeIndexException {
-        return RTreeUtils.createRTree(harness.getBufferCache(), typeTraits,
-                valueProviderFactories, rtreeCmpFactories);
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType) throws TreeIndexException {
+        return RTreeUtils.createRTree(harness.getBufferCache(), harness.getFileMapProvider(), typeTraits,
+                valueProviderFactories, rtreeCmpFactories, rtreePolicyType, harness.getFileReference());
 
     }
 
     @Override
-    protected ITreeIndexTestWorkerFactory getWorkerFactory() {
+    protected IIndexTestWorkerFactory getWorkerFactory() {
         return workerFactory;
     }
 
@@ -66,31 +72,29 @@
 
         // Insert only workload.
         TestOperation[] insertOnlyOps = new TestOperation[] { TestOperation.INSERT };
-        workloadConfs.add(new TestWorkloadConf(insertOnlyOps, getUniformOpProbs(insertOnlyOps)));
+        workloadConfs.add(new TestWorkloadConf(insertOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertOnlyOps.length)));
 
         // Inserts mixed with scans.
         TestOperation[] insertSearchOnlyOps = new TestOperation[] { TestOperation.INSERT, TestOperation.SCAN,
                 TestOperation.DISKORDER_SCAN };
-        workloadConfs.add(new TestWorkloadConf(insertSearchOnlyOps, getUniformOpProbs(insertSearchOnlyOps)));
+        workloadConfs.add(new TestWorkloadConf(insertSearchOnlyOps, ProbabilityHelper
+                .getUniformProbDist(insertSearchOnlyOps.length)));
 
         // Inserts and deletes.
         TestOperation[] insertDeleteOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE };
-        workloadConfs.add(new TestWorkloadConf(insertDeleteOps, getUniformOpProbs(insertDeleteOps)));
+        workloadConfs.add(new TestWorkloadConf(insertDeleteOps, ProbabilityHelper
+                .getUniformProbDist(insertDeleteOps.length)));
 
         // All operations mixed.
         TestOperation[] allOps = new TestOperation[] { TestOperation.INSERT, TestOperation.DELETE, TestOperation.SCAN,
                 TestOperation.DISKORDER_SCAN };
-        workloadConfs.add(new TestWorkloadConf(allOps, getUniformOpProbs(allOps)));
+        workloadConfs.add(new TestWorkloadConf(allOps, ProbabilityHelper.getUniformProbDist(allOps.length)));
 
         return workloadConfs;
     }
 
     @Override
-    protected int getFileId() {
-        return harness.getTreeFileId();
-    }
-
-    @Override
     protected String getIndexTypeName() {
         return "RTree";
     }
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 f5867e6..3d0381b 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
@@ -19,11 +19,11 @@
 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.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.AbstractTreeIndexTestWorker;
+import edu.uci.ics.hyracks.storage.am.common.AbstractIndexTestWorker;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 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.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
@@ -31,14 +31,14 @@
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
 
-public class RTreeTestWorker extends AbstractTreeIndexTestWorker {
+public class RTreeTestWorker extends AbstractIndexTestWorker {
 
     private final RTree rtree;
     private final int numFields;
     private final ArrayTupleReference rearrangedTuple = new ArrayTupleReference();
     private final ArrayTupleBuilder rearrangedTb;
 
-    public RTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, ITreeIndex index, int numBatches) {
+    public RTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
         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 d4f14ca..36b47bb 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,16 +15,16 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree.multithread;
 
-import edu.uci.ics.hyracks.storage.am.common.AbstractTreeIndexTestWorker;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestWorkerFactory;
+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;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
 
-public class RTreeTestWorkerFactory implements ITreeIndexTestWorkerFactory {
+public class RTreeTestWorkerFactory implements IIndexTestWorkerFactory {
     @Override
-    public AbstractTreeIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
-            ITreeIndex index, int numBatches) {
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
+            IIndex index, int numBatches) {
         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 039fb0b..8c1f0aa 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,13 +18,16 @@
 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.io.FileReference;
 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;
 import edu.uci.ics.hyracks.storage.am.rtree.AbstractRTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 @SuppressWarnings("rawtypes")
 public class RTreeTestContext extends AbstractRTreeTestContext {
@@ -35,25 +38,24 @@
 
     @Override
     public int getKeyFieldCount() {
-        RTree rtree = (RTree) treeIndex;
+        RTree rtree = (RTree) index;
         return rtree.getComparatorFactories().length;
     }
 
     @Override
     public IBinaryComparatorFactory[] getComparatorFactories() {
-        RTree rtree = (RTree) treeIndex;
+        RTree rtree = (RTree) index;
         return rtree.getComparatorFactories();
     }
 
-    public static RTreeTestContext create(IBufferCache bufferCache, int rtreeFileId,
-            ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
-            int numKeyFields) throws Exception {
+    public static RTreeTestContext create(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            FileReference file, ISerializerDeserializer[] fieldSerdes,
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType)
+            throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
-        RTree rtree = RTreeUtils
-                .createRTree(bufferCache, typeTraits, valueProviderFactories, cmpFactories);
-        rtree.create(rtreeFileId);
-        rtree.open(rtreeFileId);
+        RTree rtree = RTreeUtils.createRTree(bufferCache, fileMapProvider, typeTraits, valueProviderFactories,
+                cmpFactories, rtreePolicyType, file);
         RTreeTestContext testCtx = new RTreeTestContext(fieldSerdes, rtree);
         return testCtx;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestHarness.java
index c0cec35..e324a7f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/utils/RTreeTestHarness.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
@@ -31,10 +32,6 @@
 public class RTreeTestHarness {
 
     private static final long RANDOM_SEED = 50;
-    private static final int DEFAULT_PAGE_SIZE = 256;
-    private static final int DEFAULT_NUM_PAGES = 1000;
-    private static final int DEFAULT_MAX_OPEN_FILES = 10;
-    private static final int DEFAULT_HYRACKS_FRAME_SIZE = 128;
 
     protected final int pageSize;
     protected final int numPages;
@@ -43,6 +40,7 @@
 
     protected IHyracksTaskContext ctx;
     protected IBufferCache bufferCache;
+    protected IFileMapProvider fileMapProvider;
     protected int treeFileId;
 
     protected final Random rnd = new Random();
@@ -50,12 +48,13 @@
     protected final String tmpDir = System.getProperty("java.io.tmpdir");
     protected final String sep = System.getProperty("file.separator");
     protected String fileName;
+    protected FileReference file;
 
     public RTreeTestHarness() {
-        this.pageSize = DEFAULT_PAGE_SIZE;
-        this.numPages = DEFAULT_NUM_PAGES;
-        this.maxOpenFiles = DEFAULT_MAX_OPEN_FILES;
-        this.hyracksFrameSize = DEFAULT_HYRACKS_FRAME_SIZE;
+        this.pageSize = AccessMethodTestsConfig.RTREE_PAGE_SIZE;
+        this.numPages = AccessMethodTestsConfig.RTREE_NUM_PAGES;
+        this.maxOpenFiles = AccessMethodTestsConfig.RTREE_MAX_OPEN_FILES;
+        this.hyracksFrameSize = AccessMethodTestsConfig.RTREE_HYRACKS_FRAME_SIZE;
     }
 
     public RTreeTestHarness(int pageSize, int numPages, int maxOpenFiles, int hyracksFrameSize) {
@@ -67,19 +66,15 @@
 
     public void setUp() throws HyracksDataException {
         fileName = tmpDir + sep + simpleDateFormat.format(new Date());
+        file = new FileReference(new File(fileName));
         ctx = TestUtils.create(getHyracksFrameSize());
         TestStorageManagerComponentHolder.init(pageSize, numPages, maxOpenFiles);
         bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
-        IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
-        FileReference file = new FileReference(new File(fileName));
-        bufferCache.createFile(file);
-        treeFileId = fmp.lookupFileId(file);
-        bufferCache.openFile(treeFileId);
+        fileMapProvider = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
         rnd.setSeed(RANDOM_SEED);
     }
 
     public void tearDown() throws HyracksDataException {
-        bufferCache.closeFile(treeFileId);
         bufferCache.close();
         File f = new File(fileName);
         f.deleteOnExit();
@@ -93,8 +88,8 @@
         return bufferCache;
     }
 
-    public int getTreeFileId() {
-        return treeFileId;
+    public IFileMapProvider getFileMapProvider() {
+        return fileMapProvider;
     }
 
     public String getFileName() {
@@ -120,4 +115,8 @@
     public int getMaxOpenFiles() {
         return maxOpenFiles;
     }
+
+    public FileReference getFileReference() {
+        return file;
+    }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml b/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
index 8e429f9..98dbc7c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
+++ b/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
@@ -1,8 +1,6 @@
 <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>hyracks-storage-common-test</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <name>hyracks-storage-common-test</name>
 
   <parent>
diff --git a/hyracks/hyracks-tests/pom.xml b/hyracks/hyracks-tests/pom.xml
index 054ec68..bd5839f 100644
--- a/hyracks/hyracks-tests/pom.xml
+++ b/hyracks/hyracks-tests/pom.xml
@@ -1,8 +1,6 @@
 <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>hyracks-tests</artifactId>
-  <version>0.2.3-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>hyracks-tests</name>
 
@@ -15,7 +13,11 @@
   <modules>
     <module>hyracks-storage-common-test</module>
     <module>hyracks-storage-am-btree-test</module>
-    <module>hyracks-storage-am-invertedindex-test</module>
     <module>hyracks-storage-am-rtree-test</module>
+    <module>hyracks-storage-am-lsm-common-test</module>
+    <module>hyracks-storage-am-lsm-btree-test</module>
+    <module>hyracks-storage-am-lsm-rtree-test</module>
+    <module>hyracks-storage-am-lsm-invertedindex-test</module>
+    <module>hyracks-storage-am-bloomfilter-test</module>
   </modules>
 </project>
diff --git a/hyracks/pom.xml b/hyracks/pom.xml
index 9c324df..1592384 100644
--- a/hyracks/pom.xml
+++ b/hyracks/pom.xml
@@ -8,7 +8,7 @@
   <name>hyracks</name>
 
   <properties>
-    <jvm.extraargs/>
+    <jvm.extraargs />
   </properties>
 
   <build>
@@ -18,21 +18,21 @@
         <artifactId>maven-release-plugin</artifactId>
         <version>2.0</version>
         <configuration>
-            <goals>package source:jar javadoc:jar deploy:deploy</goals>
+          <goals>package source:jar javadoc:jar deploy:deploy</goals>
         </configuration>
       </plugin>
       <plugin>
-      	<groupId>org.codehaus.mojo</groupId>
-      	<artifactId>versions-maven-plugin</artifactId>
-      	<version>1.2</version>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>versions-maven-plugin</artifactId>
+        <version>1.2</version>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
         <version>2.12</version>
         <configuration>
-            <forkMode>pertest</forkMode>
-            <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs}</argLine>
+          <forkMode>pertest</forkMode>
+          <argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs} -Xmx2048m</argLine>
         </configuration>
       </plugin>
     </plugins>
@@ -83,8 +83,12 @@
     <module>hyracks-data</module>
     <module>hyracks-storage-common</module>
     <module>hyracks-storage-am-common</module>
+    <module>hyracks-storage-am-bloomfilter</module>
     <module>hyracks-storage-am-btree</module>
-    <module>hyracks-storage-am-invertedindex</module>
+    <module>hyracks-storage-am-lsm-invertedindex</module>
+    <module>hyracks-storage-am-lsm-common</module>
+    <module>hyracks-storage-am-lsm-btree</module>
+    <module>hyracks-storage-am-lsm-rtree</module>
     <module>hyracks-storage-am-rtree</module>
     <module>hyracks-test-support</module>
     <module>hyracks-tests</module>
@@ -92,9 +96,9 @@
     <module>hyracks-examples</module>
     <module>hyracks-documentation</module>
     <module>hyracks-hadoop-compat</module>
-    <!--module>hyracks-yarn</module-->
     <module>hyracks-maven-plugins</module>
     <module>hyracks-hdfs</module>
     <module>hyracks-dist</module>
+    <!--module>hyracks-yarn</module -->
   </modules>
 </project>
diff --git a/pregelix/pregelix-core/pom.xml b/pregelix/pregelix-core/pom.xml
index 2f167fb..f084614 100644
--- a/pregelix/pregelix-core/pom.xml
+++ b/pregelix/pregelix-core/pom.xml
@@ -311,5 +311,12 @@
 			<type>jar</type>
 			<scope>compile</scope>
 		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>
+				hyracks-storage-am-lsm-invertedindex
+			</artifactId>
+			<version>0.2.3-SNAPSHOT</version>
+		</dependency>
 	</dependencies>
 </project>
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 0b1be61..77fd1a7 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
@@ -52,13 +52,13 @@
 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.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+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.dataflow.TreeIndexDropOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 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;
@@ -80,8 +80,8 @@
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 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.TreeIndexRegistryProvider;
 import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdPartitionComputerFactory;
 
@@ -93,7 +93,7 @@
     protected static final String PRIMARY_INDEX = "primary";
     protected final Configuration conf;
     protected final PregelixJob giraphJob;
-    protected IIndexRegistryProvider<IIndex> treeRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
+    protected IIndexLifecycleManagerProvider lcManagerProvider = IndexLifeCycleManagerProvider.INSTANCE;
     protected IStorageManagerInterface storageManagerInterface = StorageManagerInterface.INSTANCE;
     protected String jobId = new UUID(System.currentTimeMillis(), System.nanoTime()).toString();
     protected int frameSize = ClusterConfig.getFrameSize();
@@ -169,8 +169,9 @@
 
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
         TreeIndexCreateOperatorDescriptor btreeCreate = new TreeIndexCreateOperatorDescriptor(spec,
-                storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
+                new BTreeDataflowHelperFactory(), new TransientLocalResourceFactoryProvider(),
+                NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, btreeCreate);
         return spec;
     }
@@ -229,9 +230,9 @@
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory(),
-                NoOpOperationCallbackProvider.INSTANCE);
+                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
+                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, new BTreeDataflowHelperFactory(),
+                NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
 
         /**
@@ -356,8 +357,8 @@
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
         BTreeSearchOperatorDescriptor scanner = new BTreeSearchOperatorDescriptor(spec, recordDescriptor,
-                storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                null, null, true, true, new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackProvider.INSTANCE);
+                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
+                null, null, true, true, new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -424,9 +425,10 @@
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
+
         BTreeSearchOperatorDescriptor scanner = new BTreeSearchOperatorDescriptor(spec, recordDescriptor,
-                storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                null, null, true, true, new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackProvider.INSTANCE);
+                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
+                null, null, true, true, new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
         /**
@@ -455,8 +457,8 @@
         JobSpecification spec = new JobSpecification();
 
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, indexName);
-        TreeIndexDropOperatorDescriptor drop = new TreeIndexDropOperatorDescriptor(spec, storageManagerInterface,
-                treeRegistryProvider, fileSplitProvider);
+        IndexDropOperatorDescriptor drop = new IndexDropOperatorDescriptor(spec, storageManagerInterface,
+                lcManagerProvider, fileSplitProvider, new BTreeDataflowHelperFactory());
 
         ClusterConfig.setLocationConstraint(spec, drop);
         spec.addRoot(drop);
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 9de4c04..fe2fcac 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
@@ -40,8 +40,8 @@
 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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+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;
@@ -135,7 +135,7 @@
         RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
-                recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
                 new BTreeDataflowHelperFactory(), inputRdFactory, 6,
                 new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
@@ -166,8 +166,8 @@
         indexCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration + 1,
                 WritableComparator.get(vertexIdClass).getClass());
         TreeIndexBulkReLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkReLoadOperatorDescriptor(spec,
-                storageManagerInterface, treeRegistryProvider, secondaryFileSplitProvider, typeTraits,
-                indexCmpFactories, fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory());
+                storageManagerInterface, lcManagerProvider, secondaryFileSplitProvider, typeTraits, indexCmpFactories,
+                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory());
         ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
 
         /**
@@ -222,18 +222,18 @@
          * add the insert operator to insert vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
          * add the delete operator to delete vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -343,7 +343,7 @@
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
                 typeTraits));
         IndexNestedLoopJoinOperatorDescriptor setUnion = new IndexNestedLoopJoinOperatorDescriptor(spec, rdFinal,
-                storageManagerInterface, treeRegistryProvider, secondaryFileSplitProviderRead, interiorFrameFactory,
+                storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderRead, interiorFrameFactory,
                 leafFrameFactory, typeTraits, comparatorFactories, true, keyFields, keyFields, true, true,
                 new BTreeDataflowHelperFactory(), true);
         ClusterConfig.setLocationConstraint(spec, setUnion);
@@ -361,8 +361,8 @@
                 vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
-                spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true,
+                spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+                JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true,
                 new BTreeDataflowHelperFactory(), inputRdFactory, 6, new ComputeUpdateFunctionFactory(confFactory),
                 preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete, rdFinal);
         ClusterConfig.setLocationConstraint(spec, join);
@@ -377,7 +377,7 @@
         String writeFile = iteration % 2 == 0 ? SECONDARY_INDEX_EVEN : SECONDARY_INDEX_ODD;
         IFileSplitProvider secondaryFileSplitProviderWrite = ClusterConfig.getFileSplitProvider(jobId, writeFile);
         TreeIndexBulkReLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkReLoadOperatorDescriptor(spec,
-                storageManagerInterface, treeRegistryProvider, secondaryFileSplitProviderWrite, typeTraits,
+                storageManagerInterface, lcManagerProvider, secondaryFileSplitProviderWrite, typeTraits,
                 indexCmpFactories, fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory());
         ClusterConfig.setLocationConstraint(spec, btreeBulkLoad);
 
@@ -446,18 +446,18 @@
          * add the insert operator to insert vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
          * add the delete operator to delete vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
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 91c15b2..f1eceb7 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
@@ -40,8 +40,8 @@
 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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+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;
@@ -129,7 +129,7 @@
         RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
-                recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
                 new BTreeDataflowHelperFactory(), inputRdFactory, 5,
                 new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
@@ -205,18 +205,18 @@
          */
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
          * add the delete operator to delete vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -335,7 +335,7 @@
                 vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
-                spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, interiorFrameFactory,
+                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,
@@ -408,18 +408,18 @@
          */
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
          * add the delete operator to delete vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
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 ee1fd0f..314c393 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
@@ -39,8 +39,8 @@
 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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+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;
@@ -131,7 +131,7 @@
         RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
-                recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
                 new BTreeDataflowHelperFactory(), inputRdFactory, 5,
                 new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
@@ -197,18 +197,18 @@
          */
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
          * add the delete operator to delete vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -323,7 +323,7 @@
                 vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
-                spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, interiorFrameFactory,
+                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,
@@ -384,18 +384,18 @@
 
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
          * add the delete operator to delete vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
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 628e9ce..0c3db38 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
@@ -39,8 +39,8 @@
 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.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+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;
@@ -128,7 +128,7 @@
         RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
 
         BTreeSearchFunctionUpdateOperatorDescriptor scanner = new BTreeSearchFunctionUpdateOperatorDescriptor(spec,
-                recordDescriptor, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
+                recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
                 comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
                 new BTreeDataflowHelperFactory(), inputRdFactory, 5,
                 new StartComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
@@ -211,18 +211,18 @@
          */
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
          * add the delete operator to delete vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -337,7 +337,7 @@
                 vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
-                spec, storageManagerInterface, treeRegistryProvider, fileSplitProvider, interiorFrameFactory,
+                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,
@@ -417,18 +417,18 @@
          */
         int[] fieldPermutation = new int[] { 0, 1 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.INSERT, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, insertOp);
 
         /**
          * add the delete operator to delete vertexes
          */
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, treeRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, fieldPermutation, IndexOp.DELETE, new BTreeDataflowHelperFactory(), null,
-                NoOpOperationCallbackProvider.INSTANCE);
+                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
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 572bff9..3c00cad 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
@@ -50,14 +50,14 @@
 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.IIndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
 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.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 import edu.uci.ics.pregelix.core.data.TypeTraits;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
 import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
@@ -65,8 +65,8 @@
 import edu.uci.ics.pregelix.dataflow.VertexWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.ProjectOperatorDescriptor;
+import edu.uci.ics.pregelix.runtime.bootstrap.IndexLifeCycleManagerProvider;
 import edu.uci.ics.pregelix.runtime.bootstrap.StorageManagerInterface;
-import edu.uci.ics.pregelix.runtime.bootstrap.TreeIndexRegistryProvider;
 
 public class JoinTest {
     private final static String ACTUAL_RESULT_DIR = "actual";
@@ -82,7 +82,7 @@
     private static final String PATH_TO_CLUSTER_PROPERTIES = "src/test/resources/cluster/cluster.properties";
 
     private static final float DEFAULT_BTREE_FILL_FACTOR = 1.00f;
-    private IIndexRegistryProvider<IIndex> treeRegistry = TreeIndexRegistryProvider.INSTANCE;
+    private IIndexLifecycleManagerProvider lcManagerProvider = IndexLifeCycleManagerProvider.INSTANCE;
     private IStorageManagerInterface storageManagerInterface = StorageManagerInterface.INSTANCE;
 
     private IBinaryHashFunctionFactory stringHashFactory = new PointableBinaryHashFunctionFactory(
@@ -193,8 +193,8 @@
         FileSplit[] results = new FileSplit[1];
         results[0] = resultFile;
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
-        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
-                null);
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider,
+                null, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
 
@@ -232,8 +232,9 @@
         for (int i = 0; i < typeTraits.length; i++)
             typeTraits[i] = new TypeTraits(false);
         TreeIndexCreateOperatorDescriptor writer = new TreeIndexCreateOperatorDescriptor(spec, storageManagerInterface,
-                treeRegistry, fileSplitProvider, typeTraits, comparatorFactories, new BTreeDataflowHelperFactory(),
-                NoOpOperationCallbackProvider.INSTANCE);
+                lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
+                new BTreeDataflowHelperFactory(), new TransientLocalResourceFactoryProvider(),
+                NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, NC1_ID, NC2_ID);
         spec.addRoot(writer);
         runTest(spec);
@@ -276,9 +277,9 @@
         for (int i = 0; i < typeTraits.length; i++)
             typeTraits[i] = new TypeTraits(false);
         TreeIndexBulkLoadOperatorDescriptor writer = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManagerInterface, treeRegistry, fileSplitProvider, typeTraits, comparatorFactories,
-                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory(),
-                NoOpOperationCallbackProvider.INSTANCE);
+                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
+                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, new BTreeDataflowHelperFactory(),
+                NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, NC1_ID, NC2_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, sorter, 0);
@@ -351,8 +352,8 @@
         for (int i = 0; i < typeTraits.length; i++)
             typeTraits[i] = new TypeTraits(false);
         IndexNestedLoopJoinOperatorDescriptor join = new IndexNestedLoopJoinOperatorDescriptor(spec, custOrderJoinDesc,
-                storageManagerInterface, treeRegistry, fileSplitProvider, typeTraits, keyComparatorFactories, true,
-                keyFields, keyFields, true, true, new BTreeDataflowHelperFactory());
+                storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, keyComparatorFactories,
+                true, keyFields, keyFields, true, true, new BTreeDataflowHelperFactory());
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         /** results (already in sorted order) */
@@ -360,8 +361,8 @@
         FileSplit[] results = new FileSplit[1];
         results[0] = resultFile;
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
-        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
-                null);
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider,
+                null, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
 
@@ -457,8 +458,8 @@
         FileSplit[] results = new FileSplit[1];
         results[0] = resultFile;
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
-        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
-                null);
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider,
+                null, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
 
@@ -554,7 +555,7 @@
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(new TypeAwareTupleWriterFactory(
                 typeTraits));
         IndexNestedLoopJoinOperatorDescriptor join = new IndexNestedLoopJoinOperatorDescriptor(spec, custOrderJoinDesc,
-                storageManagerInterface, treeRegistry, fileSplitProvider, interiorFrameFactory, leafFrameFactory,
+                storageManagerInterface, lcManagerProvider, fileSplitProvider, interiorFrameFactory, leafFrameFactory,
                 typeTraits, keyComparatorFactories, true, keyFields, keyFields, true, true,
                 new BTreeDataflowHelperFactory(), true, nullWriterFactories);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
@@ -564,8 +565,8 @@
         FileSplit[] results = new FileSplit[1];
         results[0] = resultFile;
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
-        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider, null,
-                null);
+        VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, null, resultFileSplitProvider,
+                null, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { NC1_ID });
         PartitionConstraintHelper.addPartitionCountConstraint(spec, writer, 1);
 
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/BTreeSearchFunctionUpdateOperatorDescriptor.java
index 99e55f1..c9f3fe7 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/BTreeSearchFunctionUpdateOperatorDescriptor.java
@@ -23,12 +23,12 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
@@ -53,14 +53,16 @@
     private final int outputArity;
 
     public BTreeSearchFunctionUpdateOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory,
             IRecordDescriptorFactory inputRdFactory, int outputArity, IUpdateFunctionFactory functionFactory,
             IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory, RecordDescriptor... rDescs) {
-        super(spec, 1, outputArity, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, null, false, NoOpOperationCallbackProvider.INSTANCE);
+        super(spec, 1, outputArity, recDesc, storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, dataflowHelperFactory, null, false,
+                new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE,
+                NoOpOperationCallbackFactory.INSTANCE);
         this.isForward = isForward;
         this.lowKeyFields = lowKeyFields;
         this.highKeyFields = highKeyFields;
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/BTreeSearchFunctionUpdateOperatorNodePushable.java
index 3938613..ff95e52 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/BTreeSearchFunctionUpdateOperatorNodePushable.java
@@ -36,9 +36,10 @@
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+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;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
@@ -106,11 +107,11 @@
          * open the function
          */
         functionProxy.functionOpen();
-        accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexHelper.getTaskContext().getFrameSize(), recDesc);
 
         try {
-            treeIndexHelper.init(false);
-            btree = (BTree) treeIndexHelper.getIndex();
+            treeIndexHelper.open();
+            btree = (BTree) treeIndexHelper.getIndexInstance();
             cursorFrame = btree.getLeafFrameFactory().createFrame();
             setCursor();
 
@@ -120,17 +121,17 @@
             rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
 
-            writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
+            writeBuffer = treeIndexHelper.getTaskContext().allocateFrame();
             tb = new ArrayTupleBuilder(btree.getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(treeIndexHelper.getHyracksTaskContext().getFrameSize());
+            appender = new FrameTupleAppender(treeIndexHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
-            indexAccessor = btree.createAccessor();
+            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
 
             cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
             updateBuffer.setFieldCount(btree.getFieldCount());
         } catch (Exception e) {
-            treeIndexHelper.deinit();
+            treeIndexHelper.close();
             throw new HyracksDataException(e);
         }
     }
@@ -203,7 +204,7 @@
              */
             functionProxy.functionClose();
         } finally {
-            treeIndexHelper.deinit();
+            treeIndexHelper.close();
         }
     }
 
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 60559e8..7662aa8 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
@@ -25,13 +25,13 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 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.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
@@ -61,14 +61,16 @@
     private final int outputArity;
 
     public IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(JobSpecification spec,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> treeIndexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
             IRecordDescriptorFactory inputRdFactory, int outputArity, IUpdateFunctionFactory functionFactory,
             IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory, RecordDescriptor... rDescs) {
-        super(spec, 1, outputArity, rDescs[0], storageManager, treeIndexRegistryProvider, fileSplitProvider,
-                typeTraits, comparatorFactories, opHelperFactory, null, false, NoOpOperationCallbackProvider.INSTANCE);
+        super(spec, 1, outputArity, rDescs[0], storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, highKeyFields, opHelperFactory, null, false,
+                new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE,
+                NoOpOperationCallbackFactory.INSTANCE);
         this.isForward = isForward;
         this.lowKeyFields = lowKeyFields;
         this.highKeyFields = highKeyFields;
@@ -88,7 +90,7 @@
     }
 
     public IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(JobSpecification spec,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> treeIndexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
             ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
@@ -96,8 +98,10 @@
             boolean isRightOuter, INullWriterFactory[] nullWriterFactories, IRecordDescriptorFactory inputRdFactory,
             int outputArity, IUpdateFunctionFactory functionFactory, IRuntimeHookFactory preHookFactory,
             IRuntimeHookFactory postHookFactory, RecordDescriptor... rDescs) {
-        super(spec, 1, outputArity, rDescs[0], storageManager, treeIndexRegistryProvider, fileSplitProvider,
-                typeTraits, comparatorFactories, opHelperFactory, null, false, NoOpOperationCallbackProvider.INSTANCE);
+        super(spec, 1, outputArity, rDescs[0], storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, highKeyFields, opHelperFactory, null, false,
+                new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE,
+                NoOpOperationCallbackFactory.INSTANCE);
         this.isForward = isForward;
         this.lowKeyFields = lowKeyFields;
         this.highKeyFields = highKeyFields;
@@ -120,7 +124,7 @@
     }
 
     public IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(JobSpecification spec,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> treeIndexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
             ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
@@ -128,8 +132,10 @@
             boolean isSetUnion, IRecordDescriptorFactory inputRdFactory, int outputArity,
             IUpdateFunctionFactory functionFactory, IRuntimeHookFactory preHookFactory,
             IRuntimeHookFactory postHookFactory, RecordDescriptor... rDescs) {
-        super(spec, 1, outputArity, rDescs[0], storageManager, treeIndexRegistryProvider, fileSplitProvider,
-                typeTraits, comparatorFactories, opHelperFactory, null, false, NoOpOperationCallbackProvider.INSTANCE);
+        super(spec, 1, outputArity, rDescs[0], storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, highKeyFields, opHelperFactory, null, false,
+                new TransientLocalResourceFactoryProvider(), NoOpOperationCallbackFactory.INSTANCE,
+                NoOpOperationCallbackFactory.INSTANCE);
         this.isForward = isForward;
         this.lowKeyFields = lowKeyFields;
         this.highKeyFields = highKeyFields;
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 37029f3..61e4649 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
@@ -36,9 +36,10 @@
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+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;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
@@ -104,12 +105,11 @@
          * open the function
          */
         functionProxy.functionOpen();
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
 
         try {
-            treeIndexOpHelper.init(false);
-            btree = (BTree) treeIndexOpHelper.getIndex();
-            btree.open(treeIndexOpHelper.getIndexFileId());
+            treeIndexOpHelper.open();
+            btree = (BTree) treeIndexOpHelper.getIndexInstance();
             cursorFrame = btree.getLeafFrameFactory().createFrame();
             setCursor();
 
@@ -140,15 +140,15 @@
 
             rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
-            writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
-            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
+            writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
+            appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor();
+            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
             updateBuffer.setFieldCount(btree.getFieldCount());
         } catch (Exception e) {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
             throw new HyracksDataException(e);
         }
     }
@@ -220,7 +220,7 @@
              */
             functionProxy.functionClose();
         } finally {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
         }
     }
 
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 ed177e3..d237761 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
@@ -24,13 +24,13 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 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.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 
 public class IndexNestedLoopJoinOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
@@ -51,12 +51,13 @@
     private boolean isSetUnion = false;
 
     public IndexNestedLoopJoinOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> treeIndexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory) {
-        super(spec, 1, 1, recDesc, storageManager, treeIndexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, opHelperFactory, null, false, NoOpOperationCallbackProvider.INSTANCE);
+        super(spec, 1, 1, recDesc, storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
+                NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
         this.isForward = isForward;
         this.lowKeyFields = lowKeyFields;
         this.highKeyFields = highKeyFields;
@@ -65,14 +66,15 @@
     }
 
     public IndexNestedLoopJoinOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> treeIndexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
             ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
             boolean isRightOuter, INullWriterFactory[] nullWriterFactories) {
-        super(spec, 1, 1, recDesc, storageManager, treeIndexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, opHelperFactory, null, false, NoOpOperationCallbackProvider.INSTANCE);
+        super(spec, 1, 1, recDesc, storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
+                NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
         this.isForward = isForward;
         this.lowKeyFields = lowKeyFields;
         this.highKeyFields = highKeyFields;
@@ -84,14 +86,15 @@
     }
 
     public IndexNestedLoopJoinOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> treeIndexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
             ITreeIndexFrameFactory leafFrameFactory, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory opHelperFactory,
             boolean isSetUnion) {
-        super(spec, 1, 1, recDesc, storageManager, treeIndexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, opHelperFactory, null, false, NoOpOperationCallbackProvider.INSTANCE);
+        super(spec, 1, 1, recDesc, storageManager, lcManagerProvider, fileSplitProvider, typeTraits,
+                comparatorFactories, null, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
+                NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
         this.isForward = isForward;
         this.lowKeyFields = lowKeyFields;
         this.highKeyFields = highKeyFields;
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 bd076d3..8b9bfc2 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
@@ -34,9 +34,10 @@
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+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;
@@ -86,11 +87,11 @@
 
     @Override
     public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
 
         try {
-            treeIndexOpHelper.init(false);
-            btree = (BTree) treeIndexOpHelper.getIndex();
+            treeIndexOpHelper.open();
+            btree = (BTree) treeIndexOpHelper.getIndexInstance();
             writer.open();
 
             int lowKeySearchFields = btree.getComparatorFactories().length;
@@ -118,15 +119,15 @@
 
             rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
-            writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+            writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
             tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
+            appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
-            indexAccessor = btree.createAccessor();
+            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             setCursor();
         } catch (Exception e) {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
             throw new HyracksDataException(e);
         }
     }
@@ -196,7 +197,7 @@
                 throw new HyracksDataException(e);
             }
         } finally {
-            treeIndexOpHelper.deinit();
+            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 f7b3d62..5ca5382 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
@@ -38,9 +38,10 @@
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+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;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
@@ -116,11 +117,11 @@
          * function open
          */
         functionProxy.functionOpen();
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
 
         try {
-            treeIndexOpHelper.init(false);
-            btree = (BTree) treeIndexOpHelper.getIndex();
+            treeIndexOpHelper.open();
+            btree = (BTree) treeIndexOpHelper.getIndexInstance();
             cursorFrame = btree.getLeafFrameFactory().createFrame();
             setCursor();
 
@@ -147,7 +148,7 @@
 
             rangePred = new RangePredicate(null, null, true, true, lowKeySearchCmp, highKeySearchCmp);
 
-            writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+            writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
 
             nullTupleBuilder = new ArrayTupleBuilder(inputRecDesc.getFields().length);
             dos = nullTupleBuilder.getDataOutput();
@@ -157,10 +158,10 @@
                 nullTupleBuilder.addFieldEndOffset();
             }
 
-            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
+            appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor();
+            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
 
             /** set the search cursor */
             rangePred.setLowKey(null, true);
@@ -178,7 +179,7 @@
             cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
             updateBuffer.setFieldCount(btree.getFieldCount());
         } catch (Exception e) {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
             throw new HyracksDataException(e);
         }
     }
@@ -273,7 +274,7 @@
         } catch (Exception e) {
             throw new HyracksDataException(e);
         } finally {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
         }
     }
 
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 9f1e1ad..d7c5d1f 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
@@ -38,9 +38,10 @@
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+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 {
@@ -97,10 +98,10 @@
 
     @Override
     public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
         try {
-            treeIndexOpHelper.init(false);
-            btree = (BTree) treeIndexOpHelper.getIndex();
+            treeIndexOpHelper.open();
+            btree = (BTree) treeIndexOpHelper.getIndexInstance();
             cursorFrame = btree.getLeafFrameFactory().createFrame();
             setCursor();
             writer.open();
@@ -129,13 +130,13 @@
 
             rangePred = new RangePredicate(null, null, true, true, lowKeySearchCmp, highKeySearchCmp);
 
-            writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+            writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
             tb = new ArrayTupleBuilder(inputRecDesc.getFields().length + btree.getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
+            appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor();
+            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
 
             /** set the search cursor */
             rangePred.setLowKey(null, true);
@@ -151,7 +152,7 @@
             }
 
         } catch (Exception e) {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
             throw new HyracksDataException(e);
         }
     }
@@ -243,7 +244,7 @@
         } catch (Exception e) {
             throw new HyracksDataException(e);
         } finally {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
         }
     }
 
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 6af60a8..160324e 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
@@ -36,9 +36,10 @@
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+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;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
@@ -104,11 +105,11 @@
     @Override
     public void open() throws HyracksDataException {
         functionProxy.functionOpen();
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
 
         try {
-            treeIndexOpHelper.init(false);
-            btree = (BTree) treeIndexOpHelper.getIndex();
+            treeIndexOpHelper.open();
+            btree = (BTree) treeIndexOpHelper.getIndexInstance();
             cursorFrame = btree.getLeafFrameFactory().createFrame();
             setCursor();
 
@@ -120,11 +121,11 @@
             }
             lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
 
-            writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
-            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
+            writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
+            appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor();
+            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
 
             /** set the search cursor */
             rangePred.setLowKey(null, true);
@@ -141,7 +142,7 @@
             cloneUpdateTb = new ArrayTupleBuilder(btree.getFieldCount());
             updateBuffer.setFieldCount(btree.getFieldCount());
         } catch (Exception e) {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
             throw new HyracksDataException(e);
         }
     }
@@ -214,7 +215,7 @@
         } catch (Exception e) {
             throw new HyracksDataException(e);
         } finally {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
         }
     }
 
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 615a25b..579935b 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
@@ -37,9 +37,10 @@
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+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;
@@ -90,11 +91,11 @@
 
     @Override
     public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
 
         try {
-            treeIndexOpHelper.init(false);
-            btree = (BTree) treeIndexOpHelper.getIndex();
+            treeIndexOpHelper.open();
+            btree = (BTree) treeIndexOpHelper.getIndexInstance();
             cursorFrame = btree.getLeafFrameFactory().createFrame();
             setCursor();
             writer.open();
@@ -107,13 +108,13 @@
             }
             lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
 
-            writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+            writeBuffer = treeIndexOpHelper.getTaskContext().allocateFrame();
             tb = new ArrayTupleBuilder(btree.getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
+            appender = new FrameTupleAppender(treeIndexOpHelper.getTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            indexAccessor = btree.createAccessor();
+            indexAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
 
             /** set the search cursor */
             rangePred.setLowKey(null, true);
@@ -129,7 +130,7 @@
             }
 
         } catch (Exception e) {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
             throw new HyracksDataException(e);
         }
     }
@@ -203,7 +204,7 @@
         } catch (Exception e) {
             throw new HyracksDataException(e);
         } finally {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
         }
     }
 
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 126fcb8..eb5ece6 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
@@ -22,12 +22,12 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 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.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 
 public class TreeIndexBulkReLoadOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
@@ -35,20 +35,21 @@
 
     private final int[] fieldPermutation;
     private final IStorageManagerInterface storageManager;
-    private final IIndexRegistryProvider<IIndex> treeIndexRegistryProvider;
+    private final IIndexLifecycleManagerProvider lcManagerProvider;
     private final IFileSplitProvider fileSplitProvider;
     private final float fillFactor;
 
     public TreeIndexBulkReLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<IIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider,
+            IIndexLifecycleManagerProvider lcManagerProvider, IFileSplitProvider fileSplitProvider,
             ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
             float fillFactor, IIndexDataflowHelperFactory opHelperFactory) {
-        super(spec, 1, 0, null, storageManager, treeIndexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, opHelperFactory, null, false, NoOpOperationCallbackProvider.INSTANCE);
+        super(spec, 1, 0, null, storageManager, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
+                fieldPermutation, opHelperFactory, null, false, new TransientLocalResourceFactoryProvider(),
+                NoOpOperationCallbackFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE);
         this.fieldPermutation = fieldPermutation;
 
         this.storageManager = storageManager;
-        this.treeIndexRegistryProvider = treeIndexRegistryProvider;
+        this.lcManagerProvider = lcManagerProvider;
         this.fileSplitProvider = fileSplitProvider;
         this.fillFactor = fillFactor;
     }
@@ -57,6 +58,6 @@
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new TreeIndexBulkReLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor,
-                recordDescProvider, storageManager, treeIndexRegistryProvider, fileSplitProvider);
+                recordDescProvider, storageManager, lcManagerProvider, fileSplitProvider);
     }
 }
\ No newline at end of file
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 883fef4..5e089a5 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
@@ -20,111 +20,54 @@
 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;
-import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 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.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
+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.tuples.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class TreeIndexBulkReLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
-    private final TreeIndexDataflowHelper treeIndexOpHelper;
-    private FrameTupleAccessor accessor;
-    private IIndexBulkLoadContext bulkLoadCtx;
-
-    private IRecordDescriptorProvider recordDescProvider;
-    private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
-
-    private final IStorageManagerInterface storageManager;
-    private final IIndexRegistryProvider<IIndex> treeIndexRegistryProvider;
-    private final IFileSplitProvider fileSplitProvider;
-    private final int partition;
     private final float fillFactor;
-    private IHyracksTaskContext ctx;
+    private final TreeIndexDataflowHelper treeIndexOpHelper;
+    private final IIndexOperatorDescriptor opDesc;
+    private final IRecordDescriptorProvider recordDescProvider;
+    private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
+
     private ITreeIndex index;
+    private FrameTupleAccessor accessor;
+    private IIndexBulkLoader bulkLoader;
 
     public TreeIndexBulkReLoadOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, int[] fieldPermutation, float fillFactor, IRecordDescriptorProvider recordDescProvider,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> treeIndexRegistryProvider,
+            IStorageManagerInterface storageManager, IIndexLifecycleManagerProvider lcManagerProvider,
             IFileSplitProvider fileSplitProvider) {
+        this.fillFactor = fillFactor;
         treeIndexOpHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
+        this.opDesc = opDesc;
         this.recordDescProvider = recordDescProvider;
         tuple.setFieldPermutation(fieldPermutation);
-
-        this.storageManager = storageManager;
-        this.treeIndexRegistryProvider = treeIndexRegistryProvider;
-        this.fileSplitProvider = fileSplitProvider;
-        this.partition = partition;
-        this.ctx = ctx;
-        this.fillFactor = fillFactor;
     }
 
     @Override
     public void open() throws HyracksDataException {
-        initDrop();
-        init();
-    }
-
-    private void initDrop() throws HyracksDataException {
-        try {
-            IndexRegistry<IIndex> treeIndexRegistry = treeIndexRegistryProvider.getRegistry(ctx);
-            IBufferCache bufferCache = storageManager.getBufferCache(ctx);
-            IFileMapProvider fileMapProvider = storageManager.getFileMapProvider(ctx);
-
-            FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
-            int indexFileId = -1;
-            boolean fileIsMapped = false;
-            synchronized (fileMapProvider) {
-                fileIsMapped = fileMapProvider.isMapped(f);
-                if (fileIsMapped)
-                    indexFileId = fileMapProvider.lookupFileId(f);
-            }
-
-            /**
-             * delete the file if it is mapped
-             */
-            if (fileIsMapped) {
-                // Unregister tree instance.
-                synchronized (treeIndexRegistry) {
-                    treeIndexRegistry.unregister(indexFileId);
-                }
-
-                // remove name to id mapping
-                bufferCache.deleteFile(indexFileId, false);
-            }
-        }
-        // TODO: for the time being we don't throw,
-        // with proper exception handling (no hanging job problem) we should
-        // throw
-        catch (Exception e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-    private void init() throws HyracksDataException {
-        AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
-                .getOperatorDescriptor();
         RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getTaskContext().getFrameSize(), recDesc);
+        treeIndexOpHelper.create();
+        treeIndexOpHelper.open();
         try {
-            treeIndexOpHelper.init(true);
-            treeIndexOpHelper.getIndex().open(treeIndexOpHelper.getIndexFileId());
-            index = (ITreeIndex) treeIndexOpHelper.getIndex();
-            index.open(treeIndexOpHelper.getIndexFileId());
-            bulkLoadCtx = index.beginBulkLoad(fillFactor);
+            index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
+            bulkLoader = index.createBulkLoader(fillFactor, false, 0);
         } catch (Exception e) {
             // cleanup in case of failure
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
             throw new HyracksDataException(e);
         }
     }
@@ -135,16 +78,22 @@
         int tupleCount = accessor.getTupleCount();
         for (int i = 0; i < tupleCount; i++) {
             tuple.reset(accessor, i);
-            index.bulkLoadAddTuple(tuple, bulkLoadCtx);
+            try {
+                bulkLoader.add(tuple);
+            } catch (IndexException e) {
+                throw new HyracksDataException(e);
+            }
         }
     }
 
     @Override
     public void close() throws HyracksDataException {
         try {
-            index.endBulkLoad(bulkLoadCtx);
+            bulkLoader.end();
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
         } finally {
-            treeIndexOpHelper.deinit();
+            treeIndexOpHelper.close();
         }
     }
 
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 567e220..8d6ab38 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
@@ -28,8 +28,8 @@
 import edu.uci.ics.hyracks.api.io.FileReference;
 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.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+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.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -38,20 +38,25 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.storage.common.smi.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceRepository;
 import edu.uci.ics.pregelix.api.graph.Vertex;
 
 public class RuntimeContext implements IWorkspaceFileFactory {
     private static final Logger LOGGER = Logger.getLogger(RuntimeContext.class.getName());
 
-    private IndexRegistry<IIndex> treeIndexRegistry;
-    private IBufferCache bufferCache;
-    private IFileMapManager fileMapManager;
-    private Map<StateKey, IStateObject> appStateMap = new ConcurrentHashMap<StateKey, IStateObject>();
-    private Map<String, Long> giraphJobIdToSuperStep = new ConcurrentHashMap<String, Long>();
-    private Map<String, Boolean> giraphJobIdToMove = new ConcurrentHashMap<String, Boolean>();
-    private IOManager ioManager;
-    private Map<Long, List<FileReference>> iterationToFiles = new ConcurrentHashMap<Long, List<FileReference>>();
+    private final IIndexLifecycleManager lcManager;
+    private final ILocalResourceRepository localResourceRepository;
+    private final ResourceIdFactory resourceIdFactory;
+    private final IBufferCache bufferCache;
+    private final IFileMapManager fileMapManager;
+    private final Map<StateKey, IStateObject> appStateMap = new ConcurrentHashMap<StateKey, IStateObject>();
+    private final Map<String, Long> giraphJobIdToSuperStep = new ConcurrentHashMap<String, Long>();
+    private final Map<String, Boolean> giraphJobIdToMove = new ConcurrentHashMap<String, Boolean>();
+    private final IOManager ioManager;
+    private final Map<Long, List<FileReference>> iterationToFiles = new ConcurrentHashMap<Long, List<FileReference>>();
 
     public RuntimeContext(INCApplicationContext appCtx) {
         fileMapManager = new TransientFileMapManager();
@@ -64,8 +69,10 @@
         /** let the buffer cache never flush dirty pages */
         bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), allocator, prs,
                 new PreDelayPageCleanerPolicy(Long.MAX_VALUE), fileMapManager, pageSize, numPages, 1000000);
-        treeIndexRegistry = new IndexRegistry<IIndex>();
         ioManager = (IOManager) appCtx.getRootContext().getIOManager();
+        lcManager = new IndexLifecycleManager();
+        localResourceRepository = new TransientLocalResourceRepository();
+        resourceIdFactory = new ResourceIdFactory(0);
     }
 
     public void close() {
@@ -80,6 +87,18 @@
         System.gc();
     }
 
+    public ILocalResourceRepository getLocalResourceRepository() {
+        return localResourceRepository;
+    }
+
+    public ResourceIdFactory getResourceIdFactory() {
+        return resourceIdFactory;
+    }
+
+    public IIndexLifecycleManager getIndexLifecycleManager() {
+        return lcManager;
+    }
+
     public IBufferCache getBufferCache() {
         return bufferCache;
     }
@@ -88,10 +107,6 @@
         return fileMapManager;
     }
 
-    public IndexRegistry<IIndex> getTreeIndexRegistry() {
-        return treeIndexRegistry;
-    }
-
     public Map<StateKey, IStateObject> getAppStateStore() {
         return appStateMap;
     }
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/IndexLifeCycleManagerProvider.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/IndexLifeCycleManagerProvider.java
new file mode 100644
index 0000000..4fce6b3
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/IndexLifeCycleManagerProvider.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.pregelix.runtime.bootstrap;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.pregelix.dataflow.context.RuntimeContext;
+
+public class IndexLifeCycleManagerProvider implements IIndexLifecycleManagerProvider {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IIndexLifecycleManagerProvider INSTANCE = new IndexLifeCycleManagerProvider();
+
+    private IndexLifeCycleManagerProvider() {
+    }
+
+    @Override
+    public IIndexLifecycleManager getLifecycleManager(IHyracksTaskContext ctx) {
+        return RuntimeContext.get(ctx).getIndexLifecycleManager();
+    }
+
+}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/StorageManagerInterface.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/StorageManagerInterface.java
index 57bbfbe..0cce59d 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/StorageManagerInterface.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/StorageManagerInterface.java
@@ -18,6 +18,8 @@
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 import edu.uci.ics.pregelix.dataflow.context.RuntimeContext;
 
 public class StorageManagerInterface implements IStorageManagerInterface {
@@ -37,4 +39,14 @@
     public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
         return RuntimeContext.get(ctx).getFileMapManager();
     }
+
+    @Override
+    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
+        return RuntimeContext.get(ctx).getLocalResourceRepository();
+    }
+
+    @Override
+    public ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
+        return RuntimeContext.get(ctx).getResourceIdFactory();
+    }
 }
\ No newline at end of file
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/TreeIndexRegistryProvider.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/TreeIndexRegistryProvider.java
deleted file mode 100644
index 7d66422..0000000
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/TreeIndexRegistryProvider.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Copyright 2009-2010 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.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-import edu.uci.ics.pregelix.dataflow.context.RuntimeContext;
-
-public class TreeIndexRegistryProvider implements IIndexRegistryProvider<IIndex> {
-    private static final long serialVersionUID = 1L;
-
-    public static final TreeIndexRegistryProvider INSTANCE = new TreeIndexRegistryProvider();
-
-    private TreeIndexRegistryProvider() {
-    }
-
-    @Override
-    public IndexRegistry<IIndex> getRegistry(IHyracksTaskContext ctx) {
-        return RuntimeContext.get(ctx).getTreeIndexRegistry();
-    }
-}
\ No newline at end of file