Merge branch 'master' into vinayakb/resource_management
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
index e000b85..5fd6ef4 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
 
 import java.util.ArrayList;
-import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
@@ -22,13 +21,8 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
-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.IPartitioningProperty;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -45,13 +39,13 @@
     private final IDataSourceIndex<?, ?> dataSourceIndex;
 
     public IndexInsertDeletePOperator(List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
-    		Mutable<ILogicalExpression> filterExpr, IDataSourceIndex<?, ?> dataSourceIndex) {
+            Mutable<ILogicalExpression> filterExpr, IDataSourceIndex<?, ?> dataSourceIndex) {
         this.primaryKeys = primaryKeys;
         this.secondaryKeys = secondaryKeys;
         if (filterExpr != null) {
-        	this.filterExpr = filterExpr.getValue();
+            this.filterExpr = filterExpr.getValue();
         } else {
-        	this.filterExpr = null;
+            this.filterExpr = null;
         }
         this.dataSourceIndex = dataSourceIndex;
     }
@@ -64,7 +58,7 @@
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();        
+        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
     }
 
     @Override
@@ -73,15 +67,11 @@
         List<LogicalVariable> scanVariables = new ArrayList<LogicalVariable>();
         scanVariables.addAll(primaryKeys);
         scanVariables.add(new LogicalVariable(-1));
-        IPartitioningProperty pp = dataSourceIndex.getDataSource().getPropertiesProvider()
-                .computePropertiesVector(scanVariables).getPartitioningProperty();
-        List<ILocalStructuralProperty> orderProps = new LinkedList<ILocalStructuralProperty>();
-        for (LogicalVariable k : secondaryKeys) {
-            orderProps.add(new LocalOrderProperty(new OrderColumn(k, OrderKind.ASC)));
-        }
-        StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(pp,
-                orderProps) };
-        return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+        IPhysicalPropertiesVector r = dataSourceIndex.getDataSource().getPropertiesProvider()
+                .computePropertiesVector(scanVariables);
+        IPhysicalPropertiesVector[] requirements = new IPhysicalPropertiesVector[1];
+        requirements[0] = r;
+        return new PhysicalRequirements(requirements, IPartitioningRequirementsCoordinator.NO_COORDINATION);
     }
 
     @SuppressWarnings({ "rawtypes", "unchecked" })
@@ -93,8 +83,8 @@
         IMetadataProvider mp = context.getMetadataProvider();
 
         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;
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(insertDeleteOp);
diff --git a/algebricks/algebricks-examples/piglet-example/pom.xml b/algebricks/algebricks-examples/piglet-example/pom.xml
index 4b101d7..254d016 100644
--- a/algebricks/algebricks-examples/piglet-example/pom.xml
+++ b/algebricks/algebricks-examples/piglet-example/pom.xml
@@ -48,6 +48,24 @@
 					</includes>
 				</configuration>
 			</plugin>
+                        <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-source</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>${project.build.directory}/generated-sources/javacc/</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
 		</plugins>
 		<pluginManagement>
 			<plugins>
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
index fa5000e..06172d9 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
@@ -55,6 +55,11 @@
             return false;
         }
 
+        //stop rewriting if the operators originates from a nested tuple source
+        if (insideSubplan(opRef)) {
+            return false;
+        }
+
         // We may pull selects above the join we create in order to eliminate possible dependencies between
         // the outer and inner input plans of the join.
         List<ILogicalOperator> topSelects = new ArrayList<ILogicalOperator>();
@@ -286,4 +291,24 @@
         return findPlanPartition((AbstractLogicalOperator) op.getInputs().get(0).getValue(), innerUsedVars,
                 outerUsedVars, innerOps, outerOps, topSelects, belowSecondUnnest);
     }
+
+    /**
+     * check whether the operator is inside a sub-plan
+     * 
+     * @param nestedRootRef
+     * @return true-if it is; false otherwise.
+     */
+    private boolean insideSubplan(Mutable<ILogicalOperator> nestedRootRef) {
+        AbstractLogicalOperator nestedRoot = (AbstractLogicalOperator) nestedRootRef.getValue();
+        if (nestedRoot.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+            return true;
+        }
+        List<Mutable<ILogicalOperator>> inputs = nestedRoot.getInputs();
+        for (Mutable<ILogicalOperator> input : inputs) {
+            if (insideSubplan(input)) {
+                return true;
+            }
+        }
+        return false;
+    }
 }
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
index f017e0f..9becf6e 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
@@ -48,57 +48,46 @@
 
 /**
  * Factors out common sub-expressions by assigning them to a variables, and replacing the common sub-expressions with references to those variables.
- *
  * Preconditions/Assumptions:
  * Assumes no projects are in the plan. This rule ignores variable reference expressions and constants (other rules deal with those separately).
- * 
  * Postconditions/Examples:
  * Plan with extracted sub-expressions. Generates one assign operator per extracted expression.
- * 
  * Example 1 - Simple Arithmetic Example (simplified)
- * 
  * Before plan:
  * assign [$$1] <- [5 + 6 - 10]
- *   assign [$$0] <- [5 + 6 + 30]
- * 
+ * assign [$$0] <- [5 + 6 + 30]
  * After plan:
  * assign [$$1] <- [$$5 - 10]
- *   assign [$$0] <- [$$5 + 30]
- *     assign [$$5] <- [5 + 6]
- * 
+ * assign [$$0] <- [$$5 + 30]
+ * assign [$$5] <- [5 + 6]
  * Example 2 - Cleaning up 'Distinct By' (simplified)
- * 
  * Before plan: (notice how $$0 is not live after the distinct)
  * assign [$$3] <- [field-access($$0, 1)]
- *   distinct ([%0->$$5])
- *     assign [$$5] <- [field-access($$0, 1)]
- *       unnest $$0 <- [scan-dataset]
- * 
+ * distinct ([%0->$$5])
+ * assign [$$5] <- [field-access($$0, 1)]
+ * unnest $$0 <- [scan-dataset]
  * After plan: (notice how the issue of $$0 is fixed)
  * assign [$$3] <- [$$5]
- *   distinct ([$$5])
- *     assign [$$5] <- [field-access($$0, 1)]
- *       unnest $$0 <- [scan-dataset]
- * 
+ * distinct ([$$5])
+ * assign [$$5] <- [field-access($$0, 1)]
+ * unnest $$0 <- [scan-dataset]
  * Example 3 - Pulling Common Expressions Above Joins (simplified)
- * 
  * Before plan:
  * assign [$$9] <- funcZ(funcY($$8))
- *   join (funcX(funcY($$8)))
- * 
+ * join (funcX(funcY($$8)))
  * After plan:
  * assign [$$9] <- funcZ($$10))
- *   select (funcX($$10))
- *     assign [$$10] <- [funcY($$8)]
- *       join (TRUE)
+ * select (funcX($$10))
+ * assign [$$10] <- [funcY($$8)]
+ * join (TRUE)
  */
 public class ExtractCommonExpressionsRule implements IAlgebraicRewriteRule {
 
     private final List<ILogicalExpression> originalAssignExprs = new ArrayList<ILogicalExpression>();
-    
+
     private final CommonExpressionSubstitutionVisitor substVisitor = new CommonExpressionSubstitutionVisitor();
     private final Map<ILogicalExpression, ExprEquivalenceClass> exprEqClassMap = new HashMap<ILogicalExpression, ExprEquivalenceClass>();
-    
+
     // Set of operators for which common subexpression elimination should not be performed.
     private static final Set<LogicalOperatorTag> ignoreOps = new HashSet<LogicalOperatorTag>();
     static {
@@ -109,9 +98,10 @@
         ignoreOps.add(LogicalOperatorTag.AGGREGATE);
         ignoreOps.add(LogicalOperatorTag.RUNNINGAGGREGATE);
     }
-    
+
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         return false;
     }
 
@@ -126,7 +116,8 @@
         return modified;
     }
 
-    private void updateEquivalenceClassMap(LogicalVariable lhs, Mutable<ILogicalExpression> rhsExprRef, ILogicalExpression rhsExpr, ILogicalOperator op) {
+    private void updateEquivalenceClassMap(LogicalVariable lhs, Mutable<ILogicalExpression> rhsExprRef,
+            ILogicalExpression rhsExpr, ILogicalOperator op) {
         ExprEquivalenceClass exprEqClass = exprEqClassMap.get(rhsExpr);
         if (exprEqClass == null) {
             exprEqClass = new ExprEquivalenceClass(op, rhsExprRef);
@@ -141,7 +132,7 @@
         if (context.checkIfInDontApplySet(this, opRef.getValue())) {
             return false;
         }
-        
+
         boolean modified = false;
         // Recurse into children.
         for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
@@ -149,7 +140,7 @@
                 modified = true;
             }
         }
-        
+
         // TODO: Deal with replicate properly. Currently, we just clear the expr equivalence map, since we want to avoid incorrect expression replacement
         // (the resulting new variables should be assigned live below a replicate).
         if (op.getOperatorTag() == LogicalOperatorTag.REPLICATE) {
@@ -160,7 +151,7 @@
         if (ignoreOps.contains(op.getOperatorTag())) {
             return modified;
         }
-        
+
         // Remember a copy of the original assign expressions, so we can add them to the equivalence class map
         // after replacing expressions within the assign operator itself.
         if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
@@ -173,13 +164,13 @@
                 originalAssignExprs.add(expr.cloneExpression());
             }
         }
-        
+
         // Perform common subexpression elimination.
         substVisitor.setOperator(op);
         if (op.acceptExpressionTransform(substVisitor)) {
             modified = true;
         }
-        
+
         // Update equivalence class map.
         if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
             AssignOperator assignOp = (AssignOperator) op;
@@ -194,7 +185,7 @@
                 // Update equivalence class map.
                 LogicalVariable lhs = assignOp.getVariables().get(i);
                 updateEquivalenceClassMap(lhs, exprRef, exprRef.getValue(), op);
-                
+
                 // Update equivalence class map with original assign expression.
                 updateEquivalenceClassMap(lhs, exprRef, originalAssignExprs.get(i), op);
             }
@@ -225,35 +216,30 @@
     }
 
     private class CommonExpressionSubstitutionVisitor implements ILogicalExpressionReferenceTransform {
-                
-        private final Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
-        private final List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+
         private IOptimizationContext context;
-        private ILogicalOperator op;        
-        
+        private ILogicalOperator op;
+
         public void setContext(IOptimizationContext context) {
             this.context = context;
         }
-        
+
         public void setOperator(ILogicalOperator op) throws AlgebricksException {
             this.op = op;
-            liveVars.clear();
-            usedVars.clear();
         }
-        
+
         @Override
         public boolean transform(Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
-            if (liveVars.isEmpty() && usedVars.isEmpty()) {
-                VariableUtilities.getLiveVariables(op, liveVars);
-                VariableUtilities.getUsedVariables(op, usedVars);
-            }
-            
             AbstractLogicalExpression expr = (AbstractLogicalExpression) exprRef.getValue();
             boolean modified = false;
             ExprEquivalenceClass exprEqClass = exprEqClassMap.get(expr);
             if (exprEqClass != null) {
                 // Replace common subexpression with existing variable. 
                 if (exprEqClass.variableIsSet()) {
+                    Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
+                    List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+                    VariableUtilities.getLiveVariables(op, liveVars);
+                    VariableUtilities.getUsedVariables(op, usedVars);
                     // Check if the replacing variable is live at this op.
                     // However, if the op is already using variables that are not live, then a replacement may enable fixing the plan.
                     // This behavior is necessary to, e.g., properly deal with distinct by.
@@ -266,9 +252,15 @@
                     }
                 } else {
                     if (assignCommonExpression(exprEqClass, expr)) {
-                        exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
-                        // Do not descend into children since this expr has been completely replaced.
-                        return true;
+                        //re-obtain the live vars after rewriting in the method called in the if condition
+                        Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
+                        VariableUtilities.getLiveVariables(op, liveVars);
+                        //rewrite only when the variable is live
+                        if (liveVars.contains(exprEqClass.getVariable())) {
+                            exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
+                            // Do not descend into children since this expr has been completely replaced.
+                            return true;
+                        }
                     }
                 }
             } else {
@@ -278,7 +270,7 @@
                     exprEqClassMap.put(expr, exprEqClass);
                 }
             }
-            
+
             // Descend into function arguments.
             if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
@@ -290,17 +282,19 @@
             }
             return modified;
         }
-        
-        private boolean assignCommonExpression(ExprEquivalenceClass exprEqClass, ILogicalExpression expr) throws AlgebricksException {
+
+        private boolean assignCommonExpression(ExprEquivalenceClass exprEqClass, ILogicalExpression expr)
+                throws AlgebricksException {
             AbstractLogicalOperator firstOp = (AbstractLogicalOperator) exprEqClass.getFirstOperator();
             Mutable<ILogicalExpression> firstExprRef = exprEqClass.getFirstExpression();
-            if (firstOp.getOperatorTag() == LogicalOperatorTag.INNERJOIN || firstOp.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+            if (firstOp.getOperatorTag() == LogicalOperatorTag.INNERJOIN
+                    || firstOp.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
                 // Do not extract common expressions from within the same join operator.
                 if (firstOp == op) {
                     return false;
                 }
                 AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) firstOp;
-                Mutable<ILogicalExpression> joinCond = joinOp.getCondition();                
+                Mutable<ILogicalExpression> joinCond = joinOp.getCondition();
                 ILogicalExpression enclosingExpr = getEnclosingExpression(joinCond, firstExprRef.getValue());
                 if (enclosingExpr == null) {
                     // No viable enclosing expression that we can pull out from the join.
@@ -312,12 +306,13 @@
                 op.getInputs().get(0).setValue(selectOp);
                 // Set firstOp to be the select below op, since we want to assign the common subexpr there.
                 firstOp = (AbstractLogicalOperator) selectOp;
-            } else if (firstOp.getInputs().size() > 1) { 
+            } else if (firstOp.getInputs().size() > 1) {
                 // Bail for any non-join operator with multiple inputs.
                 return false;
-            }                        
+            }
             LogicalVariable newVar = context.newVar();
-            AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(firstExprRef.getValue().cloneExpression()));            
+            AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(firstExprRef
+                    .getValue().cloneExpression()));
             // Place assign below firstOp.
             newAssign.getInputs().add(new MutableObject<ILogicalOperator>(firstOp.getInputs().get(0).getValue()));
             newAssign.setExecutionMode(firstOp.getExecutionMode());
@@ -330,7 +325,8 @@
             return true;
         }
 
-        private ILogicalExpression getEnclosingExpression(Mutable<ILogicalExpression> conditionExprRef, ILogicalExpression commonSubExpr) {
+        private ILogicalExpression getEnclosingExpression(Mutable<ILogicalExpression> conditionExprRef,
+                ILogicalExpression commonSubExpr) {
             ILogicalExpression conditionExpr = conditionExprRef.getValue();
             if (conditionExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
                 return null;
@@ -371,7 +367,7 @@
             return enclosingBoolExpr;
         }
     }
-    
+
     private boolean containsExpr(ILogicalExpression expr, ILogicalExpression searchExpr) {
         if (expr == searchExpr) {
             return true;
@@ -387,7 +383,7 @@
         }
         return false;
     }
-    
+
     private boolean isEqJoinCondition(ILogicalExpression expr) {
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
         if (funcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.EQ)) {
@@ -400,38 +396,38 @@
         }
         return false;
     }
-    
+
     private final class ExprEquivalenceClass {
         // First operator in which expression is used.
         private final ILogicalOperator firstOp;
-        
+
         // Reference to expression in first op.
         private final Mutable<ILogicalExpression> firstExprRef;
-        
+
         // Variable that this expression has been assigned to.
         private LogicalVariable var;
-        
+
         public ExprEquivalenceClass(ILogicalOperator firstOp, Mutable<ILogicalExpression> firstExprRef) {
             this.firstOp = firstOp;
             this.firstExprRef = firstExprRef;
         }
-        
+
         public ILogicalOperator getFirstOperator() {
             return firstOp;
         }
-        
+
         public Mutable<ILogicalExpression> getFirstExpression() {
             return firstExprRef;
         }
-        
+
         public void setVariable(LogicalVariable var) {
             this.var = var;
         }
-        
+
         public LogicalVariable getVariable() {
             return var;
         }
-        
+
         public boolean variableIsSet() {
             return var != null;
         }
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
index 18c483e..489167f 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceGroupByForSubplanRule.java
@@ -143,8 +143,10 @@
 
         Set<LogicalVariable> pkVars = computeGbyVars(outerNts, free, context);
         if (pkVars == null || pkVars.size() < 1) {
-            // could not group only by primary keys
-            return false;
+            // there is no non-trivial primary key, group-by keys are all live variables
+            ILogicalOperator subplanInput = subplan.getInputs().get(0).getValue();
+            pkVars = new HashSet<LogicalVariable>();
+            VariableUtilities.getLiveVariables(subplanInput, pkVars);
         }
         AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Found FD for introducing group-by: " + pkVars);
 
diff --git a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
index 8ac4e86..13574ea 100644
--- a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
+++ b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
@@ -36,7 +36,7 @@
             if (isIgnored(qFile.getName(), ignores))
                 continue;
 
-            if (qFile.isFile() && qFile.getName().startsWith("h11_")) {
+            if (qFile.isFile()) {
                 String resultFileName = hiveExtToResExt(qFile.getName());
                 File rFile = new File(PATH_TO_RESULTS + resultFileName);
                 testSuite.addTest(new OptimizerTestCase(qFile, rFile));
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
index eef4071..e24aaa7 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
@@ -55,7 +55,7 @@
 
 	<property>
 		<name>hive.algebricks.groupby.external</name>
-		<value>false</value>
+		<value>true</value>
 	</property>
 
 	<property>
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/ignore.txt b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/ignore.txt
index e70ea78..e69de29 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/ignore.txt
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/ignore.txt
@@ -1 +0,0 @@
-q16
\ No newline at end of file
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/MurmurHash3BinaryHashFunctionFamily.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/MurmurHash3BinaryHashFunctionFamily.java
deleted file mode 100644
index 760a614..0000000
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/MurmurHash3BinaryHashFunctionFamily.java
+++ /dev/null
@@ -1,63 +0,0 @@
-package edu.uci.ics.hivesterix.runtime.factory.hashfunction;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
-
-public class MurmurHash3BinaryHashFunctionFamily implements IBinaryHashFunctionFamily {
-
-    public static final IBinaryHashFunctionFamily INSTANCE = new MurmurHash3BinaryHashFunctionFamily();
-
-    private static final long serialVersionUID = 1L;
-
-    private MurmurHash3BinaryHashFunctionFamily() {
-    }
-
-    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 = (bytes[p] & 0xff) | ((bytes[p + 1] & 0xff) << 8) | ((bytes[p + 2] & 0xff) << 16)
-                            | ((bytes[p + 3] & 0xff) << 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;
-                }
-                if (remain > 0) {
-                    int k = 0;
-                    for (int i = 0; remain > 0; i += 8) {
-                        k ^= (bytes[p++] & 0xff) << i;
-                        remain--;
-                    }
-                    k *= C1;
-                    k = Integer.rotateLeft(k, 15);
-                    k *= C2;
-                    h ^= k;
-                }
-                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/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java
index e7a2e79..45302e8 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java
@@ -1,9 +1,9 @@
 package edu.uci.ics.hivesterix.runtime.provider;
 
-import edu.uci.ics.hivesterix.runtime.factory.hashfunction.MurmurHash3BinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFamilyProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+import edu.uci.ics.hyracks.data.std.accessors.MurmurHash3BinaryHashFunctionFamily;
 
 public class HiveBinaryHashFunctionFamilyProvider implements IBinaryHashFunctionFamilyProvider {
 
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index 9b8a996..5976760 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -459,7 +459,7 @@
                 case NOTIFY_TASK_FAILURE: {
                     CCNCFunctions.NotifyTaskFailureFunction ntff = (CCNCFunctions.NotifyTaskFailureFunction) fn;
                     workQueue.schedule(new TaskFailureWork(ClusterControllerService.this, ntff.getJobId(), ntff
-                            .getTaskId(), ntff.getDetails(), ntff.getDetails()));
+                            .getTaskId(), ntff.getNodeId(), ntff.getExceptions()));
                     return;
                 }
 
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index 827a71e..3a6dc26 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -78,11 +78,11 @@
 
     private JobStatus status;
 
-    private Exception exception;
+    private List<Exception> exceptions;
 
     private JobStatus pendingStatus;
 
-    private Exception pendingException;
+    private List<Exception> pendingExceptions;
 
     public JobRun(ClusterControllerService ccs, DeploymentId deploymentId, JobId jobId,
             IActivityClusterGraphGenerator acgg, EnumSet<JobFlag> jobFlags) {
@@ -124,9 +124,9 @@
         return pmm;
     }
 
-    public synchronized void setStatus(JobStatus status, Exception exception) {
+    public synchronized void setStatus(JobStatus status, List<Exception> exceptions) {
         this.status = status;
-        this.exception = exception;
+        this.exceptions = exceptions;
         notifyAll();
     }
 
@@ -134,21 +134,21 @@
         return status;
     }
 
-    public synchronized Exception getException() {
-        return exception;
+    public synchronized List<Exception> getExceptions() {
+        return exceptions;
     }
 
-    public void setPendingStatus(JobStatus status, Exception exception) {
+    public void setPendingStatus(JobStatus status, List<Exception> exceptions) {
         this.pendingStatus = status;
-        this.pendingException = exception;
+        this.pendingExceptions = exceptions;
     }
 
     public JobStatus getPendingStatus() {
         return pendingStatus;
     }
 
-    public synchronized Exception getPendingException() {
-        return pendingException;
+    public synchronized List<Exception> getPendingExceptions() {
+        return pendingExceptions;
     }
 
     public long getCreateTime() {
@@ -180,8 +180,18 @@
         while (status != JobStatus.TERMINATED && status != JobStatus.FAILURE) {
             wait();
         }
-        if (exception != null) {
-            throw new HyracksException("Job Failed", exception);
+        if (exceptions != null && !exceptions.isEmpty()) {
+            StringBuilder buffer = new StringBuilder();
+            buffer.append("Job failed on account of:\n");
+            for (Exception e : exceptions) {
+                buffer.append(e.getMessage()).append('\n');
+            }
+            HyracksException he;
+            he = new HyracksException(buffer.toString(), exceptions.get(0));
+            for (int i = 1; i < exceptions.size(); ++i) {
+                he.addSuppressed(exceptions.get(i));
+            }
+            throw he;
         }
     }
 
@@ -333,9 +343,9 @@
                                 taskAttempt.put("node-id", ta.getNodeId());
                                 taskAttempt.put("start-time", ta.getStartTime());
                                 taskAttempt.put("end-time", ta.getEndTime());
-                                String failureDetails = ta.getFailureDetails();
-                                if (failureDetails != null) {
-                                    taskAttempt.put("failure-details", failureDetails);
+                                List<Exception> exceptions = ta.getExceptions();
+                                if (exceptions != null && !exceptions.isEmpty()) {
+                                    taskAttempt.put("failure-details", exceptions);
                                 }
                                 taskAttempts.put(taskAttempt);
                             }
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
index 7c0dd57..b323501 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.cc.job;
 
+import java.util.List;
+
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 
 public class TaskAttempt {
@@ -35,7 +37,7 @@
 
     private TaskStatus status;
 
-    private String failureDetails;
+    private List<Exception> exceptions;
 
     private long startTime;
 
@@ -73,13 +75,13 @@
         return status;
     }
 
-    public String getFailureDetails() {
-        return failureDetails;
+    public List<Exception> getExceptions() {
+        return exceptions;
     }
 
-    public void setStatus(TaskStatus status, String details) {
+    public void setStatus(TaskStatus status, List<Exception> exceptions) {
         this.status = status;
-        this.failureDetails = details;
+        this.exceptions = exceptions;
     }
 
     public long getStartTime() {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index f3d7d34..d09b641 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -16,6 +16,7 @@
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -454,13 +455,13 @@
         }
     }
 
-    private void abortJob(Exception exception) {
+    private void abortJob(List<Exception> exceptions) {
         Set<TaskCluster> inProgressTaskClustersCopy = new HashSet<TaskCluster>(inProgressTaskClusters);
         for (TaskCluster tc : inProgressTaskClustersCopy) {
             abortTaskCluster(findLastTaskClusterAttempt(tc));
         }
         assert inProgressTaskClusters.isEmpty();
-        ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.FAILURE, exception));
+        ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.FAILURE, exceptions));
     }
 
     private void abortTaskCluster(TaskClusterAttempt tcAttempt) {
@@ -598,7 +599,7 @@
      * @param details
      *            - Cause of the failure
      */
-    public void notifyTaskFailure(TaskAttempt ta, ActivityCluster ac, String details) {
+    public void notifyTaskFailure(TaskAttempt ta, ActivityCluster ac, List<Exception> exceptions) {
         try {
             LOGGER.fine("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
             TaskAttemptId taId = ta.getTaskAttemptId();
@@ -606,13 +607,13 @@
             TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
             if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
                 LOGGER.fine("Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
-                ta.setStatus(TaskAttempt.TaskStatus.FAILED, details);
+                ta.setStatus(TaskAttempt.TaskStatus.FAILED, exceptions);
                 abortTaskCluster(lastAttempt);
                 lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
                 lastAttempt.setEndTime(System.currentTimeMillis());
                 abortDoomedTaskClusters();
                 if (lastAttempt.getAttempt() >= jobRun.getActivityClusterGraph().getMaxReattempts()) {
-                    abortJob(new HyracksException(details));
+                    abortJob(exceptions);
                     return;
                 }
                 startRunnableActivityClusters();
@@ -621,7 +622,7 @@
                         + lastAttempt);
             }
         } catch (Exception e) {
-            abortJob(e);
+            abortJob(Collections.singletonList(e));
         }
     }
 
@@ -646,7 +647,10 @@
                             for (TaskAttempt ta : lastTaskClusterAttempt.getTaskAttempts().values()) {
                                 assert (ta.getStatus() == TaskAttempt.TaskStatus.COMPLETED || ta.getStatus() == TaskAttempt.TaskStatus.RUNNING);
                                 if (deadNodes.contains(ta.getNodeId())) {
-                                    ta.setStatus(TaskAttempt.TaskStatus.FAILED, "Node " + ta.getNodeId() + " failed");
+                                    ta.setStatus(
+                                            TaskAttempt.TaskStatus.FAILED,
+                                            Collections.singletonList(new Exception("Node " + ta.getNodeId()
+                                                    + " failed")));
                                     ta.setEndTime(System.currentTimeMillis());
                                     abort = true;
                                 }
@@ -661,7 +665,7 @@
             }
             startRunnableActivityClusters();
         } catch (Exception e) {
-            abortJob(e);
+            abortJob(Collections.singletonList(e));
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index b304b21..7954c7c 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import java.util.List;
 import java.util.Set;
 import java.util.logging.Logger;
 
@@ -36,13 +37,13 @@
     private ClusterControllerService ccs;
     private JobId jobId;
     private JobStatus status;
-    private Exception exception;
+    private List<Exception> exceptions;
 
-    public JobCleanupWork(ClusterControllerService ccs, JobId jobId, JobStatus status, Exception exception) {
+    public JobCleanupWork(ClusterControllerService ccs, JobId jobId, JobStatus status, List<Exception> exceptions) {
         this.ccs = ccs;
         this.jobId = jobId;
         this.status = status;
-        this.exception = exception;
+        this.exceptions = exceptions;
     }
 
     @Override
@@ -58,7 +59,7 @@
         }
         Set<String> targetNodes = run.getParticipatingNodeIds();
         run.getCleanupPendingNodeIds().addAll(targetNodes);
-        run.setPendingStatus(status, exception);
+        run.setPendingStatus(status, exceptions);
         if (targetNodes != null && !targetNodes.isEmpty()) {
             for (String n : targetNodes) {
                 NodeControllerState ncs = ccs.getNodeMap().get(n);
@@ -77,7 +78,7 @@
                     e.printStackTrace();
                 }
             }
-            run.setStatus(run.getPendingStatus(), run.getPendingException());
+            run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
             ccs.getActiveRunMap().remove(jobId);
             ccs.getRunMapArchive().put(jobId, run);
             try {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index 7ecdd16..0e739ba 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import java.util.Collections;
 import java.util.EnumSet;
 
 import edu.uci.ics.hyracks.api.deployment.DeploymentId;
@@ -62,7 +63,8 @@
             try {
                 run.getScheduler().startJob();
             } catch (Exception e) {
-                ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, e));
+                ccs.getWorkQueue().schedule(
+                        new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, Collections.singletonList(e)));
             }
             callback.setValue(jobId);
         } catch (Exception e) {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index ed58c43..65e1519 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -67,7 +67,7 @@
                     e.printStackTrace();
                 }
             }
-            run.setStatus(run.getPendingStatus(), run.getPendingException());
+            run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
             ccs.getActiveRunMap().remove(jobId);
             ccs.getRunMapArchive().put(jobId, run);
             try {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
index bc8c314..51eb671 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import java.util.List;
+
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.job.ActivityCluster;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -22,11 +24,12 @@
 import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
 
 public class TaskFailureWork extends AbstractTaskLifecycleWork {
-    private final String details;
+    private final List<Exception> exceptions;
 
-    public TaskFailureWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId, String details) {
+    public TaskFailureWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
+            List<Exception> exceptions) {
         super(ccs, jobId, taId, nodeId);
-        this.details = details;
+        this.exceptions = exceptions;
     }
 
     @Override
@@ -34,7 +37,7 @@
         JobRun run = ccs.getActiveRunMap().get(jobId);
         ccs.getDatasetDirectoryService().reportJobFailure(jobId);
         ActivityCluster ac = ta.getTask().getTaskCluster().getActivityCluster();
-        run.getScheduler().notifyTaskFailure(ta, ac, details);
+        run.getScheduler().notifyTaskFailure(ta, ac, exceptions);
     }
 
     @Override
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 47a5c09..9954dc3 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -37,7 +37,8 @@
     public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
             throws Exception;
 
-    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, String details) throws Exception;
+    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, List<Exception> exceptions)
+            throws Exception;
 
     public void notifyJobletCleanup(JobId jobId, String nodeId) throws Exception;
 
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
index e343657..a6382de 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -213,13 +213,13 @@
         private final JobId jobId;
         private final TaskAttemptId taskId;
         private final String nodeId;
-        private final String details;
+        private final List<Exception> exceptions;
 
-        public NotifyTaskFailureFunction(JobId jobId, TaskAttemptId taskId, String nodeId, String details) {
+        public NotifyTaskFailureFunction(JobId jobId, TaskAttemptId taskId, String nodeId, List<Exception> exceptions) {
             this.jobId = jobId;
             this.taskId = taskId;
             this.nodeId = nodeId;
-            this.details = details;
+            this.exceptions = exceptions;
         }
 
         @Override
@@ -239,8 +239,8 @@
             return nodeId;
         }
 
-        public String getDetails() {
-            return details;
+        public List<Exception> getExceptions() {
+            return exceptions;
         }
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index 5ed65cc..240322c 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -59,9 +59,9 @@
     }
 
     @Override
-    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, String details) throws Exception {
+    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, List<Exception> exceptions) throws Exception {
         CCNCFunctions.NotifyTaskFailureFunction fn = new CCNCFunctions.NotifyTaskFailureFunction(jobId, taskId, nodeId,
-                details);
+                exceptions);
         ipcHandle.send(-1, fn, null);
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index ab9098b..d0e52e3 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -14,13 +14,12 @@
  */
 package edu.uci.ics.hyracks.control.nc;
 
-import java.io.ByteArrayOutputStream;
-import java.io.PrintWriter;
-import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Hashtable;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Executor;
@@ -81,11 +80,7 @@
 
     private IOperatorNodePushable operator;
 
-    private volatile boolean failed;
-
-    private ByteArrayOutputStream errorBaos;
-
-    private PrintWriter errorWriter;
+    private final List<Exception> exceptions;
 
     private volatile boolean aborted;
 
@@ -102,9 +97,7 @@
         opEnv = joblet.getEnvironment();
         partitionSendProfile = new Hashtable<PartitionId, PartitionProfile>();
         pendingThreads = new LinkedHashSet<Thread>();
-        failed = false;
-        errorBaos = new ByteArrayOutputStream();
-        errorWriter = new PrintWriter(errorBaos, true);
+        exceptions = new ArrayList<>();
         this.ncs = ncs;
     }
 
@@ -252,10 +245,7 @@
                                     pushFrames(collector, writer);
                                 } catch (HyracksDataException e) {
                                     synchronized (Task.this) {
-                                        failed = true;
-                                        errorWriter.println("Exception caught by thread: " + thread.getName());
-                                        e.printStackTrace(errorWriter);
-                                        errorWriter.println();
+                                        exceptions.add(e);
                                     }
                                 } finally {
                                     thread.setName(oldName);
@@ -277,23 +267,15 @@
             NodeControllerService ncs = joblet.getNodeController();
             ncs.getWorkQueue().schedule(new NotifyTaskCompleteWork(ncs, this));
         } catch (Exception e) {
-            failed = true;
-            errorWriter.println("Exception caught by thread: " + ct.getName());
-            e.printStackTrace(errorWriter);
-            errorWriter.println();
+            exceptions.add(e);
         } finally {
             ct.setName(threadName);
             close();
             removePendingThread(ct);
         }
-        if (failed) {
-            errorWriter.close();
+        if (!exceptions.isEmpty()) {
             NodeControllerService ncs = joblet.getNodeController();
-            try {
-                ncs.getWorkQueue().schedule(new NotifyTaskFailureWork(ncs, this, errorBaos.toString("UTF-8")));
-            } catch (UnsupportedEncodingException e) {
-                e.printStackTrace();
-            }
+            ncs.getWorkQueue().schedule(new NotifyTaskFailureWork(ncs, this, exceptions));
         }
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/NotifyTaskFailureWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/NotifyTaskFailureWork.java
index 013544d..c70a1bd 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/NotifyTaskFailureWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/NotifyTaskFailureWork.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.nc.work;
 
+import java.util.List;
+
 import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
@@ -23,12 +25,12 @@
 public class NotifyTaskFailureWork extends AbstractWork {
     private final NodeControllerService ncs;
     private final Task task;
-    private final String details;
+    private final List<Exception> exceptions;
 
-    public NotifyTaskFailureWork(NodeControllerService ncs, Task task, String details) {
+    public NotifyTaskFailureWork(NodeControllerService ncs, Task task, List<Exception> exceptions) {
         this.ncs = ncs;
         this.task = task;
-        this.details = details;
+        this.exceptions = exceptions;
     }
 
     @Override
@@ -39,7 +41,7 @@
             if (dpm != null) {
                 dpm.abortReader(jobId);
             }
-            ncs.getClusterController().notifyTaskFailure(jobId, task.getTaskAttemptId(), ncs.getId(), details);
+            ncs.getClusterController().notifyTaskFailure(jobId, task.getTaskAttemptId(), ncs.getId(), exceptions);
         } catch (Exception e) {
             e.printStackTrace();
         }
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
index 8f2e32c..588ad00 100644
--- 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
@@ -3,9 +3,23 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 
-public class MurmurHash3BinaryHashFunctionFamily implements IBinaryHashFunctionFamily {
+/**
+ * An implementation of the Murmur3 hash family. The code is implemented based
+ * on the original <a
+ * href=http://code.google.com/p/guava-libraries/source/browse
+ * /guava/src/com/google/common/hash/Murmur3_32HashFunction.java>guava
+ * implementation</a> from Google Guava library.
+ */
+public class MurmurHash3BinaryHashFunctionFamily implements
+        IBinaryHashFunctionFamily {
+
+    public static final IBinaryHashFunctionFamily INSTANCE = new MurmurHash3BinaryHashFunctionFamily();
+
     private static final long serialVersionUID = 1L;
 
+    private MurmurHash3BinaryHashFunctionFamily() {
+    }
+
     private static final int C1 = 0xcc9e2d51;
     private static final int C2 = 0x1b873593;
     private static final int C3 = 5;
@@ -21,9 +35,10 @@
                 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);
+                while (remain >= 4) {
+                    int k = (bytes[p] & 0xff) | ((bytes[p + 1] & 0xff) << 8)
+                            | ((bytes[p + 2] & 0xff) << 16)
+                            | ((bytes[p + 3] & 0xff) << 24);
                     k *= C1;
                     k = Integer.rotateLeft(k, 15);
                     k *= C2;
@@ -33,20 +48,16 @@
                     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;
+                if (remain > 0) {
+                    int k = 0;
+                    for (int i = 0; remain > 0; i += 8) {
+                        k ^= (bytes[p++] & 0xff) << i;
+                        remain--;
+                    }
+                    k *= C1;
+                    k = Integer.rotateLeft(k, 15);
+                    k *= C2;
+                    h ^= k;
                 }
                 h ^= length;
                 h ^= (h >>> 16);
diff --git a/hyracks/hyracks-dataflow-std/pom.xml b/hyracks/hyracks-dataflow-std/pom.xml
index 56046f7..8e67891 100644
--- a/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks/hyracks-dataflow-std/pom.xml
@@ -45,5 +45,10 @@
   		<type>jar</type>
   		<scope>test</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>commons-io</groupId>
+  		<artifactId>commons-io</artifactId>
+  		<version>2.4</version>
+  	</dependency>
   </dependencies>
 </project>
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileRemoveOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileRemoveOperatorDescriptor.java
new file mode 100644
index 0000000..bf7ff33
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileRemoveOperatorDescriptor.java
@@ -0,0 +1,81 @@
+/*
+ * 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.dataflow.std.file;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.io.FileUtils;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+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.AbstractOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+
+public class FileRemoveOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private final IFileSplitProvider fileSplitProvider;
+
+    public FileRemoveOperatorDescriptor(IOperatorDescriptorRegistry spec, IFileSplitProvider fileSplitProvder) {
+        super(spec, 0, 0);
+        this.fileSplitProvider = fileSplitProvder;
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        final FileSplit split = fileSplitProvider.getFileSplits()[partition];
+        return new AbstractOperatorNodePushable() {
+
+            @Override
+            public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                throw new IllegalStateException();
+            }
+
+            @Override
+            public void initialize() throws HyracksDataException {
+                File f = split.getLocalFile().getFile();
+                try {
+                    FileUtils.deleteDirectory(f);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public IFrameWriter getInputFrameWriter(int index) {
+                throw new IllegalStateException();
+            }
+
+            @Override
+            public int getInputArity() {
+                return 0;
+            }
+
+            @Override
+            public void deinitialize() throws HyracksDataException {
+            }
+        };
+    }
+
+}
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
index 935724b..932e166 100644
--- 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
@@ -25,6 +25,7 @@
     protected final String sep = System.getProperty("file.separator");
     protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
     protected static int DEFAULT_MEM_NUM_PAGES = 1000;
+    protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
 
     public String getPrimaryIndexName() {
         return System.getProperty("java.io.tmpdir") + sep + "primary" + simpleDateFormat.format(new Date());
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
index 62d4362..dfd5495 100644
--- 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
@@ -54,7 +54,7 @@
         invertedIndexDataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
                 new ConstantMergePolicyProvider(MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
                 SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE,
-                DEFAULT_MEM_NUM_PAGES);
+                DEFAULT_MEM_NUM_PAGES, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
     }
 
     @Override
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
index c35c6c9..9e5a47b 100644
--- 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
@@ -48,7 +48,8 @@
 
         invertedIndexDataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(new ConstantMergePolicyProvider(
                 MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
-                SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES);
+                SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE,
+                DEFAULT_MEM_NUM_PAGES, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
     }
 
     @Override
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
index 912ab0e..c67cbfd 100644
--- 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
@@ -35,7 +35,8 @@
     public IIndexDataflowHelperFactory createDataFlowHelperFactory() {
         return new LSMBTreeDataflowHelperFactory(new ConstantMergePolicyProvider(MERGE_THRESHOLD),
                 ThreadCountingOperationTrackerFactory.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
-                NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES);
+                NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
     }
 
 }
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
index 84b34b7..a892b4e 100644
--- 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
@@ -42,6 +42,6 @@
         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);
+                DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
     }
 }
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 715f822..89679ae 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
@@ -63,6 +63,7 @@
 
     protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
     protected static int DEFAULT_MEM_NUM_PAGES = 1000;
+    protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
 
     @Rule
     public TemporaryFolder outputFolder = new TemporaryFolder();
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
index 06f06c6..79f48fa 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
@@ -40,16 +40,17 @@
     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);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider);
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
     }
 
     @Override
@@ -62,7 +63,7 @@
         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);
+                        .getTreeIndexBloomFilterKeyFields(), bloomFilterFalsePositiveRate, 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
index ebf4bc0..e706786 100644
--- 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
@@ -30,16 +30,17 @@
 
     public LSMBTreeDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         super(mergePolicyProvider, opTrackerFactory, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
-                memNumPages);
+                memNumPages, bloomFilterFalsePositiveRate);
     }
 
     @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);
+                bloomFilterFalsePositiveRate, 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/LSMBTree.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 81c2367..1652f5a 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -93,12 +93,12 @@
             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);
+            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+            IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(memFreePageManager, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         mutableComponent = new LSMBTreeMutableComponent(new BTree(memBufferCache,
                 ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), memFreePageManager, interiorFrameFactory,
                 insertLeafFrameFactory, cmpFactories, fieldCount, new FileReference(new File("membtree"))),
@@ -367,7 +367,7 @@
 
         int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
         BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                bloomFilterFalsePositiveRate);
 
         LSMBTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getBTreeFlushTarget(),
                 flushOp.getBloomFilterFlushTarget(), true);
@@ -427,7 +427,7 @@
 
         int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
         BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                bloomFilterFalsePositiveRate);
         LSMBTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
                 mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
 
@@ -516,7 +516,7 @@
 
             int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
             BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                    MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                    bloomFilterFalsePositiveRate);
             builder = ((LSMBTreeImmutableComponent) component).getBloomFilter().createBuilder(numElementsHint,
                     bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
         }
@@ -647,9 +647,9 @@
             btree.validate();
         }
     }
-    
+
     @Override
     public String toString() {
-        return "LSMBTree [" + fileManager.getBaseDir() + "]"; 
+        return "LSMBTree [" + fileManager.getBaseDir() + "]";
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
index ac20b6d..9033797 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
@@ -49,20 +49,21 @@
     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) {
+            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
         return createLSMTree(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache, diskFileMapProvider,
-                typeTraits, cmpFactories, bloomFilterKeyFields, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider, 0);
+                typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, 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) {
+            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            int startIODeviceIndex) {
         LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
                 cmpFactories.length, false);
         LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
@@ -91,8 +92,8 @@
 
         LSMBTree lsmTree = new LSMBTree(memBufferCache, memFreePageManager, interiorFrameFactory,
                 insertLeafFrameFactory, deleteLeafFrameFactory, fileNameManager, diskBTreeFactory,
-                bulkLoadBTreeFactory, bloomFilterFactory, diskFileMapProvider, typeTraits.length, cmpFactories,
-                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                bulkLoadBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider,
+                typeTraits.length, cmpFactories, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         return lsmTree;
     }
 }
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
index ea7c3b4..6202518 100644
--- 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
@@ -27,9 +27,11 @@
 
     protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
     protected static int DEFAULT_MEM_NUM_PAGES = 1000;
+    protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
 
     protected final int memPageSize;
     protected final int memNumPages;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected final ILSMMergePolicy mergePolicy;
     protected final ILSMIOOperationScheduler ioScheduler;
@@ -39,17 +41,19 @@
     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);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
         super(opDesc, ctx, partition);
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.mergePolicy = mergePolicy;
         this.opTrackerFactory = opTrackerFactory;
         this.ioScheduler = ioScheduler;
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
index a2f2a11..bc4271c 100644
--- 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
@@ -30,15 +30,18 @@
     protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
     protected final int memPageSize;
     protected final int memNumPages;
+    protected final double bloomFilterFalsePositiveRate;
 
     public AbstractLSMIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         this.mergePolicyProvider = mergePolicyProvider;
         this.opTrackerFactory = opTrackerFactory;
         this.ioSchedulerProvider = ioSchedulerProvider;
         this.ioOpCallbackProvider = ioOpCallbackProvider;
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 12c7ea7..6782fbe 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -39,8 +39,6 @@
 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.01;
-
     protected final ILSMHarness lsmHarness;
 
     protected final ILSMIOOperationScheduler ioScheduler;
@@ -54,19 +52,22 @@
     protected final ILSMIndexFileManager fileManager;
     protected final IFileMapProvider diskFileMapProvider;
     protected final AtomicReference<List<ILSMComponent>> componentsRef;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected boolean isActivated;
 
     private boolean needsFlush = false;
 
     public AbstractLSMIndex(IInMemoryFreePageManager memFreePageManager, IBufferCache diskBufferCache,
-            ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ILSMMergePolicy mergePolicy,
+            ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
         this.memFreePageManager = memFreePageManager;
         this.diskBufferCache = diskBufferCache;
         this.diskFileMapProvider = diskFileMapProvider;
         this.fileManager = fileManager;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.ioScheduler = ioScheduler;
         this.ioOpCallbackProvider = ioOpCallbackProvider;
         ILSMOperationTracker opTracker = opTrackerFactory.createOperationTracker(this);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
index 3d8b391..c997f9d 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
@@ -42,16 +42,17 @@
     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);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider);
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
     }
 
     @Override
@@ -69,8 +70,8 @@
                     diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
                     invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
                     invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
-                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), mergePolicy, opTrackerFactory,
-                    ioScheduler, ioOpCallbackProvider, partition);
+                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
+                    mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, partition);
             return invIndex;
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
index 9796ebc..60a0b47 100644
--- 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
@@ -30,16 +30,18 @@
 
     public LSMInvertedIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         super(mergePolicyProvider, opTrackerProvider, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
-                memNumPages);
+                memNumPages, bloomFilterFalsePositiveRate);
     }
 
     @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);
+        return new LSMInvertedIndexDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
+                bloomFilterFalsePositiveRate, 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/PartitionedLSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
index c5b4f07..9427275 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
@@ -42,16 +42,17 @@
     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);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, 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);
+            int partition, int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
     }
 
     @Override
@@ -69,8 +70,8 @@
                     memFreePageManager, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
                     invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
                     invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
-                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), mergePolicy, opTrackerFactory,
-                    ioScheduler, ioOpCallbackProvider, partition);
+                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate,
+                    mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, partition);
             return invIndex;
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
index 8a8aad2..801462b 100644
--- 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
@@ -30,17 +30,17 @@
 
     public PartitionedLSMInvertedIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         super(mergePolicyProvider, opTrackerProvider, ioSchedulerProvider, ioOpCallbackProvider, memNumPages,
-                memNumPages);
+                memNumPages, bloomFilterFalsePositiveRate);
     }
 
     @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);
+                bloomFilterFalsePositiveRate, 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/impls/LSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 30fdd27..beb4dc9 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -97,15 +97,15 @@
 
     public LSMInvertedIndex(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
             OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, ILSMIndexFileManager fileManager,
-            IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+            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);
+        super(memFreePageManager, diskInvIndexFactory.getBufferCache(), fileManager, diskFileMapProvider,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         this.memFreePageManager = memFreePageManager;
         this.tokenizerFactory = tokenizerFactory;
         this.invListTypeTraits = invListTypeTraits;
@@ -314,6 +314,7 @@
                 throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
             }
         }
+        mutableComponent.setIsModified();
     }
 
     @Override
@@ -461,7 +462,7 @@
         if (numBTreeTuples > 0) {
             int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numBTreeTuples);
             BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                    MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                    bloomFilterFalsePositiveRate);
 
             // Create an BTree instance for the deleted keys.
             BTree diskDeletedKeysBTree = component.getDeletedKeysBTree();
@@ -736,9 +737,9 @@
             component.getDeletedKeysBTree().validate();
         }
     }
-    
+
     @Override
     public String toString() {
-        return "LSMInvertedIndex [" + fileManager.getBaseDir() + "]"; 
+        return "LSMInvertedIndex [" + fileManager.getBaseDir() + "]";
     }
 }
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
index 1b293eb..a45f729 100644
--- 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
@@ -38,15 +38,17 @@
     public PartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
             IInMemoryFreePageManager memFreePageManager, OnDiskInvertedIndexFactory diskInvIndexFactory,
             BTreeFactory deletedKeysBTreeFactory, BloomFilterFactory bloomFilterFactory,
-            ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+            double bloomFilterFalsePositiveRate, 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);
+                bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories,
+                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     protected InMemoryInvertedIndex createInMemoryInvertedIndex(IInMemoryBufferCache memBufferCache)
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index 79c8ccf..a3ff152 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -122,12 +122,13 @@
             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 {
+            String onDiskDir, double bloomFilterFalsePositiveRate, 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);
+                onDiskDir, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider, 0);
     }
 
     public static LSMInvertedIndex createLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
@@ -135,9 +136,9 @@
             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 {
+            String onDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int startIODeviceIndex) throws IndexException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
                 invListCmpFactories, diskBufferCache);
@@ -160,9 +161,9 @@
                 tokenCmpFactories, fileManager);
 
         LSMInvertedIndex invIndex = new LSMInvertedIndex(memBufferCache, memFreePageManager, invIndexFactory,
-                deletedKeysBTreeFactory, bloomFilterFactory, fileManager, diskFileMapProvider, invListTypeTraits,
-                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager,
+                diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         return invIndex;
     }
 
@@ -171,13 +172,13 @@
             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 {
+            String onDiskDir, double bloomFilterFalsePositiveRate, 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);
+                diskBufferCache, ioManager, onDiskDir, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+                ioScheduler, ioOpCallbackProvider, 0);
     }
 
     public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
@@ -185,9 +186,9 @@
             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 {
+            String onDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int startIODeviceIndex) throws IndexException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
                 invListCmpFactories, diskBufferCache);
@@ -210,9 +211,9 @@
                 tokenTypeTraits, tokenCmpFactories, fileManager);
 
         PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(memBufferCache, memFreePageManager,
-                invIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory, fileManager, diskFileMapProvider,
-                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                invIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate,
+                fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                tokenCmpFactories, tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         return invIndex;
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
index c363c99..5fcdcea 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
@@ -55,19 +55,20 @@
             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);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, btreeComparatorFactories, valueProviderFactories,
+                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
     }
 
     public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, IBinaryComparatorFactory[] btreeComparatorFactories,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate,
+            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);
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         this.btreeComparatorFactories = btreeComparatorFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
index 1df914e..49e0a23 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
@@ -50,13 +50,15 @@
     }
 
     public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, IBinaryComparatorFactory[] btreeComparatorFactories,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate,
+            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);
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, btreeComparatorFactories,
+                valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider, linearizeCmpFactory);
     }
 
     @Override
@@ -69,8 +71,8 @@
         try {
             return LSMRTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache,
                     diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
-                    rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider,
-                    linearizeCmpFactory, startIODeviceIndex);
+                    rtreePolicyType, bloomFilterFalsePositiveRate, 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
index a730895..0778e15 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
@@ -41,9 +41,10 @@
             RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
             ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
-            ILinearizeComparatorFactory linearizeCmpFactory, int memPageSize, int memNumPages) {
+            ILinearizeComparatorFactory linearizeCmpFactory, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         super(mergePolicyProvider, opTrackerFactory, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
-                memNumPages);
+                memNumPages, bloomFilterFalsePositiveRate);
         this.btreeComparatorFactories = btreeComparatorFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
@@ -53,8 +54,9 @@
     @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);
+        return new LSMRTreeDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
+                bloomFilterFalsePositiveRate, 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
index 6f5ecb1..d4fac8f 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
@@ -54,7 +54,7 @@
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
             ILinearizeComparatorFactory linearizeCmpFactory) {
-        super(opDesc, ctx, partition, memPageSize, memNumPages, btreeComparatorFactories, valueProviderFactories,
+        super(opDesc, ctx, partition, memPageSize, memNumPages, 0, btreeComparatorFactories, valueProviderFactories,
                 rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, 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
index 4b2e075..f62fab1 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -91,10 +91,11 @@
             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);
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(memFreePageManager, diskRTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         RTree memRTree = new RTree(memBufferCache, ((InMemoryBufferCache) memBufferCache).getFileMapProvider(),
                 memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, rtreeCmpFactories, fieldCount,
                 new FileReference(new File("memrtree")));
@@ -331,7 +332,7 @@
                 NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
         BTreeAccessor btreeAccessor = (BTree.BTreeAccessor) mutableComponent.getBTree().createAccessor(
                 NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        
+
         return new LSMRTreeOpContext(rtreeAccessor, (IRTreeLeafFrame) rtreeLeafFrameFactory.createFrame(),
                 (IRTreeInteriorFrame) rtreeInteriorFrameFactory.createFrame(), memFreePageManager
                         .getMetaDataFrameFactory().createFrame(), 4, btreeAccessor, btreeLeafFrameFactory,
@@ -362,9 +363,9 @@
         InMemoryBufferCache memBufferCache = (InMemoryBufferCache) mutableComponent.getRTree().getBufferCache();
         return memBufferCache.getNumPages() * memBufferCache.getPageSize();
     }
-    
+
     @Override
     public String toString() {
-        return "LSMRTree [" + fileManager.getBaseDir() + "]"; 
+        return "LSMRTree [" + fileManager.getBaseDir() + "]";
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index fc5b06d..367b387 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -72,16 +72,17 @@
             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) {
+            double bloomFilterFalsePositiveRate, 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);
+                linearizerArray, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     /**
@@ -303,7 +304,7 @@
         if (numBTreeTuples > 0) {
             int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numBTreeTuples);
             BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                    MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                    bloomFilterFalsePositiveRate);
 
             IIndexCursor btreeScanCursor = memBTreeAccessor.createSearchCursor();
             memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index be9073e..a3a8ca2 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -81,7 +81,7 @@
         super(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
                 btreeInteriorFrameFactory, btreeLeafFrameFactory, fileManager, diskRTreeFactory,
                 new LSMRTreeWithAntiMatterTuplesComponentFactory(diskRTreeFactory), diskFileMapProvider, fieldCount,
-                rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray, mergePolicy,
+                rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray, 0, mergePolicy,
                 opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         bulkLoaComponentFactory = new LSMRTreeWithAntiMatterTuplesComponentFactory(bulkLoadRTreeFactory);
         this.bTreeTupleSorter = null;
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index 6c9fce6..d6efc34 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -65,12 +65,14 @@
             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 {
+            double bloomFilterFalsePositiveRate, 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);
+                typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+                linearizeCmpFactory, 0);
     }
 
     public static LSMRTree createLSMTree(IInMemoryBufferCache memBufferCache,
@@ -78,9 +80,10 @@
             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 {
+            double bloomFilterFalsePositiveRate, 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);
 
@@ -117,9 +120,9 @@
                 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);
+                diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate,
+                diskFileMapProvider, typeTraits.length, rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory,
+                comparatorFields, linearizerArray, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         return lsmTree;
     }
 
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
index f962200..73176f3 100644
--- 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
@@ -53,6 +53,7 @@
     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;
+    public static final double LSM_RTREE_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
 
     // Mem configuration for BTree.
     public static final int BTREE_PAGE_SIZE = 256;
@@ -67,6 +68,7 @@
     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;
+    public static final double LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
 
     // Mem configuration for Inverted Index.
     public static final int LSM_INVINDEX_DISK_PAGE_SIZE = 1024;
@@ -75,6 +77,7 @@
     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;
+    public static final double LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
     // Test parameters.
     public static final int LSM_INVINDEX_NUM_DOCS_TO_INSERT = 100;
     // Used for full-fledged search test.
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
index 4bd1910..843d200 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
@@ -53,7 +53,7 @@
             BTreeLeafFrameType leafType) throws Exception {
         return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),harness.getMergePolicy(),
                 harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
index 069faad..eb97c56 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
@@ -53,8 +53,8 @@
             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.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index 539ed3e..76972a5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -37,7 +37,8 @@
         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.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
index f17e3c8..d070f23 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
@@ -53,8 +53,8 @@
             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.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
index 24d1f10..0455c20 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
@@ -44,7 +44,8 @@
         harness.setUp();
         testCtx = LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length, harness.getMergePolicy(),
+                harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
         index = testCtx.getIndex();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
index da36c79..76fe0b8 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
@@ -52,8 +52,8 @@
             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.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 648e70f..8464e9e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -46,7 +46,8 @@
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
-                harness.getMergePolicy(), NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
index 3a99c16..3c5060b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
@@ -54,8 +54,8 @@
             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.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index b59ec0d..431f68b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -39,7 +39,8 @@
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
-                harness.getMergePolicy(), NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
index ca89512..b19e21e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
@@ -53,8 +53,8 @@
             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.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index c494448..de9fe24 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -53,7 +53,8 @@
         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.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 5d2185a..5a9726b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -74,8 +74,8 @@
     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 {
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
+            double bloomFilterFalsePositiveRate) throws BTreeException, HyracksException {
         this.numBatches = numBatches;
 
         this.onDiskPageSize = onDiskPageSize;
@@ -96,7 +96,7 @@
                 new LIFOMetaDataFrameFactory());
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         lsmtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, bufferCache, fmp,
-                typeTraits, cmpFactories, bloomFilterKeyFields, NoMergePolicy.INSTANCE,
+                typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
                 ThreadCountingOperationTrackerFactory.INSTANCE, ioScheduler, NoOpIOOperationCallback.INSTANCE);
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index 8bef8f0..f0380e9 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -67,9 +67,9 @@
     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 {
+            int numKeyFields, double bloomFilterFalsePositiveRate, 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];
@@ -77,8 +77,8 @@
             bloomFilterKeyFields[i] = i;
         }
         LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
-                diskBufferCache, diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                diskBufferCache, diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields,
+                bloomFilterFalsePositiveRate, 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
index 9128607..231c8ff 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -63,6 +63,7 @@
     protected final int memPageSize;
     protected final int memNumPages;
     protected final int hyracksFrameSize;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected IOManager ioManager;
     protected IBufferCache diskBufferCache;
@@ -88,6 +89,7 @@
         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.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
@@ -95,13 +97,14 @@
     }
 
     public LSMBTreeTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
-            int memNumPages, int hyracksFrameSize) {
+            int memNumPages, int hyracksFrameSize, double bloomFilterFalsePositiveRate) {
         this.diskPageSize = diskPageSize;
         this.diskNumPages = diskNumPages;
         this.diskMaxOpenFiles = diskMaxOpenFiles;
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
         this.hyracksFrameSize = hyracksFrameSize;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
@@ -181,6 +184,10 @@
         return memBufferCache;
     }
 
+    public double getBoomFilterFalsePositiveRate() {
+        return bloomFilterFalsePositiveRate;
+    }
+
     public IInMemoryFreePageManager getMemFreePageManager() {
         return memFreePageManager;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index 5be1d6a..9c617a1 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -57,6 +57,7 @@
     protected final int memPageSize;
     protected final int memNumPages;
     protected final int hyracksFrameSize;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected IOManager ioManager;
     protected IBufferCache diskBufferCache;
@@ -84,6 +85,7 @@
         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.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
@@ -91,13 +93,14 @@
     }
 
     public LSMInvertedIndexTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
-            int memNumPages, int hyracksFrameSize) {
+            int memNumPages, int hyracksFrameSize, double bloomFilterFalsePositiveRate) {
         this.diskPageSize = diskPageSize;
         this.diskNumPages = diskNumPages;
         this.diskMaxOpenFiles = diskMaxOpenFiles;
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
         this.hyracksFrameSize = hyracksFrameSize;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
@@ -182,6 +185,10 @@
         return memBufferCache;
     }
 
+    public double getBoomFilterFalsePositiveRate() {
+        return bloomFilterFalsePositiveRate;
+    }
+
     public IInMemoryFreePageManager getMemFreePageManager() {
         return memFreePageManager;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index d39ce5d..a88a88c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -146,7 +146,8 @@
                         harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                         harness.getDiskBufferCache(), harness.getIOManager(), harness.getOnDiskDir(),
-                        harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                        harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                         harness.getIOOperationCallbackProvider());
                 break;
             }
@@ -155,7 +156,8 @@
                         harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                         harness.getDiskBufferCache(), harness.getIOManager(), harness.getOnDiskDir(),
-                        harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                        harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                         harness.getIOOperationCallbackProvider());
                 break;
             }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
index 995f18c..4dffcdd 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
@@ -57,7 +57,8 @@
         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.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
index d72b668..039c1e5 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
@@ -57,7 +57,8 @@
         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.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index cba8cee..8ebb0e4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -40,8 +40,8 @@
         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(),
+                valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
index 96485f8..0cceece 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
@@ -57,7 +57,8 @@
         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.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
index e72b3ca..18ccefb 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
@@ -55,7 +55,8 @@
         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.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
         index = testCtx.getIndex();
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
index 1d07484..a2088d1 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
@@ -56,7 +56,8 @@
         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.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
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
index cc46065..edcc69e 100644
--- 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
@@ -57,7 +57,8 @@
         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.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider());
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index 37be58d..fc8fd3e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -60,8 +60,8 @@
         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(),
+                valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index e2409db..b341752 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -72,9 +72,9 @@
             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 {
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
                 .serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -82,8 +82,9 @@
                 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));
+                valueProviderFactories, rtreePolicyType, bloomFilterFalsePositiveRate, 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
index 6fb6f9c..2270526 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -59,6 +59,7 @@
     protected final int memPageSize;
     protected final int memNumPages;
     protected final int hyracksFrameSize;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected IOManager ioManager;
     protected IBufferCache diskBufferCache;
@@ -83,6 +84,7 @@
         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.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_RTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_RTREE_HYRACKS_FRAME_SIZE;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
@@ -91,12 +93,13 @@
     }
 
     public LSMRTreeTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
-            int memNumPages, int hyracksFrameSize) {
+            int memNumPages, int hyracksFrameSize, double bloomFilterFalsePositiveRate) {
         this.diskPageSize = diskPageSize;
         this.diskNumPages = diskNumPages;
         this.diskMaxOpenFiles = diskMaxOpenFiles;
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.hyracksFrameSize = hyracksFrameSize;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
@@ -176,6 +179,10 @@
         return memBufferCache;
     }
 
+    public double getBoomFilterFalsePositiveRate() {
+        return bloomFilterFalsePositiveRate;
+    }
+
     public IInMemoryFreePageManager getMemFreePageManager() {
         return memFreePageManager;
     }
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 ce1a34d..8f0019e 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
@@ -173,6 +173,7 @@
                 new BTreeDataflowHelperFactory(), new TransientLocalResourceFactoryProvider(),
                 NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, btreeCreate);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
@@ -242,6 +243,7 @@
                 new WritableSerializerDeserializerFactory(vertexIdClass));
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sorter, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, btreeBulkLoad, 0);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
@@ -319,6 +321,7 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
                 comparatorFactories), sorter, 0, writer, 0);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
@@ -444,6 +447,7 @@
          */
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, writer, 0);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
@@ -462,6 +466,7 @@
 
         ClusterConfig.setLocationConstraint(spec, drop);
         spec.addRoot(drop);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
index e54373f..b8cd953 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
@@ -40,6 +40,7 @@
 
     private VLongWritable vid = new VLongWritable();
     private GraphMutationVertex newVertex = null;
+    private DoubleWritable msg = new DoubleWritable(0.0);
 
     @Override
     public void compute(Iterator<DoubleWritable> msgIterator) {
@@ -47,17 +48,20 @@
             if (newVertex == null) {
                 newVertex = new GraphMutationVertex();
             }
-            if (getVertexId().get() % 2 == 0 || getVertexId().get() % 3 == 0) {
-                deleteVertex(getVertexId());
-            } else {
-                vid.set(100 * getVertexId().get());
-                newVertex.setVertexId(vid);
-                newVertex.setVertexValue(getVertexValue());
-                addVertex(vid, newVertex);
+            if (getVertexId().get() < 100) {
+                if ((getVertexId().get() % 2 == 0 || getVertexId().get() % 3 == 0)) {
+                    deleteVertex(getVertexId());
+                } else {
+                    vid.set(100 * getVertexId().get());
+                    newVertex.setVertexId(vid);
+                    newVertex.setVertexValue(getVertexValue());
+                    addVertex(vid, newVertex);
+                    sendMsg(vid, msg);
+                }
             }
             voteToHalt();
         } else {
-            if (getVertexId().get() % 190 == 0) {
+            if (getVertexId().get() == 1900) {
                 deleteVertex(getVertexId());
             }
             voteToHalt();
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
index 00f6f54..7126e8c 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
@@ -19,6 +19,7 @@
 
 import junit.framework.TestCase;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
@@ -93,17 +94,18 @@
     @Test
     public void test() throws Exception {
         setUp();
-        Plan[] plans = new Plan[] { Plan.OUTER_JOIN };
+        Plan[] plans = new Plan[] { Plan.OUTER_JOIN_SORT, Plan.OUTER_JOIN, Plan.INNER_JOIN, Plan.OUTER_JOIN_SINGLE_SORT };
         for (Plan plan : plans) {
             driver.runJob(job, plan, PregelixHyracksIntegrationUtil.CC_HOST,
                     PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
+            compareResults();
         }
-        compareResults();
         tearDown();
         waitawhile();
     }
 
     private void compareResults() throws Exception {
+        FileUtils.deleteQuietly(new File(resultFileDir));
         dfs.copyToLocalFile(FileOutputFormat.getOutputPath(job), new Path(resultFileDir));
         TestUtils.compareWithResultDir(new File(expectedFileDir), new File(resultFileDir));
     }