checkpoint towards fixing diskComponents' LSN
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index b9596be..bee4906 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -45,6 +45,5 @@
     INSERT_DELETE,
     INDEX_INSERT_DELETE,
     UPDATE,
-    EXTENSION_OPERATOR,
-    EXTERNAL_DATA_ACCESS_BY_RID
+    EXTENSION_OPERATOR
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index f166f6f..c9ef2f3 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -60,6 +60,5 @@
     SINGLE_PARTITION_INVERTED_INDEX_SEARCH,
     LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH,
     PARTITIONINGSPLIT,
-    EXTENSION_OPERATOR,
-    EXTERNAL_ACCESS_BY_RID
+    EXTENSION_OPERATOR
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExternalDataAccessByRIDOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExternalDataAccessByRIDOperator.java
deleted file mode 100644
index 875c6b1..0000000
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/ExternalDataAccessByRIDOperator.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
-
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
-import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;
-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
-
-public class ExternalDataAccessByRIDOperator extends AbstractScanOperator{
-
-	private final List<Object> variableTypes; // TODO: get rid of this and
-	protected final Mutable<ILogicalExpression> expression;
-    
-    public ExternalDataAccessByRIDOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
-            List<Object> variableTypes) {
-        super(variables);
-        this.expression = expression;
-        this.variableTypes = variableTypes;
-    }
-
-    @Override
-    public LogicalOperatorTag getOperatorTag() {
-        return LogicalOperatorTag.EXTERNAL_DATA_ACCESS_BY_RID;
-    }
-
-    @Override
-    public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
-        IVariableTypeEnvironment env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMetadataProvider());
-            env.setVarType(variables.get(0), variableTypes.get(0));
-        return env;
-    }
-    
-    public List<Object> getVariableTypes() {
-        return variableTypes;
-    }
-    
-    @Override
-    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
-        return visitor.visitExternalDataAccessByRIDOperator(this, arg);
-    }
-    
-    @Override
-    public boolean isMap() {
-        return false;
-    }
-    
-    @Override
-    public VariablePropagationPolicy getVariablePropagationPolicy() {
-        return new VariablePropagationPolicy() {
-            @Override
-            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
-                    throws AlgebricksException {
-            	target.addVariable(variables.get(0));
-            }
-        };
-    }
-
-    public Mutable<ILogicalExpression> getExpressionRef() {
-        return expression;
-    }
-    
-	@Override
-	public boolean acceptExpressionTransform(
-			ILogicalExpressionReferenceTransform visitor)
-			throws AlgebricksException {
-		return visitor.transform(expression);
-	}
-}
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index c739b23..cebddee 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -50,7 +50,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -79,593 +78,572 @@
 
 public class FDsAndEquivClassesVisitor implements ILogicalOperatorVisitor<Void, IOptimizationContext> {
 
-	@Override
-	public Void visitAggregateOperator(AggregateOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		ctx.putEquivalenceClassMap(op, new HashMap<LogicalVariable, EquivalenceClass>());
-		ctx.putFDList(op, new ArrayList<FunctionalDependency>());
-		return null;
-	}
+    @Override
+    public Void visitAggregateOperator(AggregateOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        ctx.putEquivalenceClassMap(op, new HashMap<LogicalVariable, EquivalenceClass>());
+        ctx.putFDList(op, new ArrayList<FunctionalDependency>());
+        return null;
+    }
 
-	@Override
-	public Void visitAssignOperator(AssignOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		ILogicalOperator inp1 = op.getInputs().get(0).getValue();
-		Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
-		ctx.putEquivalenceClassMap(op, eqClasses);
-		List<LogicalVariable> used = new ArrayList<LogicalVariable>();
-		VariableUtilities.getUsedVariables(op, used);
-		List<FunctionalDependency> fds1 = getOrComputeFDs(inp1, ctx);
-		List<FunctionalDependency> eFds = new ArrayList<FunctionalDependency>(fds1.size());
-		for (FunctionalDependency fd : fds1) {
-			if (fd.getTail().containsAll(used)) {
-				List<LogicalVariable> hd = new ArrayList<LogicalVariable>(fd.getHead());
-				List<LogicalVariable> tl = new ArrayList<LogicalVariable>(fd.getTail());
-				tl.addAll(op.getVariables());
-				FunctionalDependency fd2 = new FunctionalDependency(hd, tl);
-				eFds.add(fd2);
-			} else {
-				eFds.add(fd);
-			}
-		}
-		ctx.putFDList(op, eFds);
-		return null;
-	}
+    @Override
+    public Void visitAssignOperator(AssignOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        ILogicalOperator inp1 = op.getInputs().get(0).getValue();
+        Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+        ctx.putEquivalenceClassMap(op, eqClasses);
+        List<LogicalVariable> used = new ArrayList<LogicalVariable>();
+        VariableUtilities.getUsedVariables(op, used);
+        List<FunctionalDependency> fds1 = getOrComputeFDs(inp1, ctx);
+        List<FunctionalDependency> eFds = new ArrayList<FunctionalDependency>(fds1.size());
+        for (FunctionalDependency fd : fds1) {
+            if (fd.getTail().containsAll(used)) {
+                List<LogicalVariable> hd = new ArrayList<LogicalVariable>(fd.getHead());
+                List<LogicalVariable> tl = new ArrayList<LogicalVariable>(fd.getTail());
+                tl.addAll(op.getVariables());
+                FunctionalDependency fd2 = new FunctionalDependency(hd, tl);
+                eFds.add(fd2);
+            } else {
+                eFds.add(fd);
+            }
+        }
+        ctx.putFDList(op, eFds);
+        return null;
+    }
 
-	@Override
-	public Void visitDataScanOperator(DataSourceScanOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		ILogicalOperator inp1 = op.getInputs().get(0).getValue();
-		Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
-		ctx.putEquivalenceClassMap(op, eqClasses);
-		List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>(getOrComputeFDs(inp1, ctx));
-		ctx.putFDList(op, fds);
-		op.getDataSource().computeFDs(op.getVariables(), fds);
-		return null;
-	}
+    @Override
+    public Void visitDataScanOperator(DataSourceScanOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        ILogicalOperator inp1 = op.getInputs().get(0).getValue();
+        Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+        ctx.putEquivalenceClassMap(op, eqClasses);
+        List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>(getOrComputeFDs(inp1, ctx));
+        ctx.putFDList(op, fds);
+        op.getDataSource().computeFDs(op.getVariables(), fds);
+        return null;
+    }
 
-	@Override
-	public Void visitDistinctOperator(DistinctOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		ILogicalOperator op0 = op.getInputs().get(0).getValue();
-		List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
-		ctx.putFDList(op, functionalDependencies);
-		for (FunctionalDependency inherited : getOrComputeFDs(op0, ctx)) {
-			boolean isCoveredByDistinctByVars = true;
-			for (LogicalVariable v : inherited.getHead()) {
-				if (!op.isDistinctByVar(v)) {
-					isCoveredByDistinctByVars = false;
-				}
-			}
-			if (isCoveredByDistinctByVars) {
-				List<LogicalVariable> newTail = new ArrayList<LogicalVariable>();
-				for (LogicalVariable v2 : inherited.getTail()) {
-					if (op.isDistinctByVar(v2)) {
-						newTail.add(v2);
-					}
-				}
-				if (!newTail.isEmpty()) {
-					List<LogicalVariable> newHead = new ArrayList<LogicalVariable>(inherited.getHead());
-					FunctionalDependency newFd = new FunctionalDependency(newHead, newTail);
-					functionalDependencies.add(newFd);
-				}
-			}
-		}
-		Set<LogicalVariable> gbySet = new HashSet<LogicalVariable>();
-		List<Mutable<ILogicalExpression>> expressions = op.getExpressions();
-		for (Mutable<ILogicalExpression> pRef : expressions) {
-			ILogicalExpression p = pRef.getValue();
-			if (p.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-				VariableReferenceExpression v = (VariableReferenceExpression) p;
-				gbySet.add(v.getVariableReference());
-			}
-		}
-		LocalGroupingProperty lgp = new LocalGroupingProperty(gbySet);
+    @Override
+    public Void visitDistinctOperator(DistinctOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        ILogicalOperator op0 = op.getInputs().get(0).getValue();
+        List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+        ctx.putFDList(op, functionalDependencies);
+        for (FunctionalDependency inherited : getOrComputeFDs(op0, ctx)) {
+            boolean isCoveredByDistinctByVars = true;
+            for (LogicalVariable v : inherited.getHead()) {
+                if (!op.isDistinctByVar(v)) {
+                    isCoveredByDistinctByVars = false;
+                }
+            }
+            if (isCoveredByDistinctByVars) {
+                List<LogicalVariable> newTail = new ArrayList<LogicalVariable>();
+                for (LogicalVariable v2 : inherited.getTail()) {
+                    if (op.isDistinctByVar(v2)) {
+                        newTail.add(v2);
+                    }
+                }
+                if (!newTail.isEmpty()) {
+                    List<LogicalVariable> newHead = new ArrayList<LogicalVariable>(inherited.getHead());
+                    FunctionalDependency newFd = new FunctionalDependency(newHead, newTail);
+                    functionalDependencies.add(newFd);
+                }
+            }
+        }
+        Set<LogicalVariable> gbySet = new HashSet<LogicalVariable>();
+        List<Mutable<ILogicalExpression>> expressions = op.getExpressions();
+        for (Mutable<ILogicalExpression> pRef : expressions) {
+            ILogicalExpression p = pRef.getValue();
+            if (p.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                VariableReferenceExpression v = (VariableReferenceExpression) p;
+                gbySet.add(v.getVariableReference());
+            }
+        }
+        LocalGroupingProperty lgp = new LocalGroupingProperty(gbySet);
 
-		Map<LogicalVariable, EquivalenceClass> equivalenceClasses = getOrComputeEqClasses(op0, ctx);
-		ctx.putEquivalenceClassMap(op, equivalenceClasses);
+        Map<LogicalVariable, EquivalenceClass> equivalenceClasses = getOrComputeEqClasses(op0, ctx);
+        ctx.putEquivalenceClassMap(op, equivalenceClasses);
 
-		lgp.normalizeGroupingColumns(equivalenceClasses, functionalDependencies);
-		Set<LogicalVariable> normSet = lgp.getColumnSet();
-		List<Mutable<ILogicalExpression>> newDistinctByList = new ArrayList<Mutable<ILogicalExpression>>();
-		for (Mutable<ILogicalExpression> p2Ref : expressions) {
-			ILogicalExpression p2 = p2Ref.getValue();
-			if (p2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-				VariableReferenceExpression var2 = (VariableReferenceExpression) p2;
-				LogicalVariable v2 = var2.getVariableReference();
-				if (normSet.contains(v2)) {
-					newDistinctByList.add(p2Ref);
-				}
-			} else {
-				newDistinctByList.add(p2Ref);
-			}
-		}
-		expressions.clear();
-		expressions.addAll(newDistinctByList);
-		return null;
-	}
+        lgp.normalizeGroupingColumns(equivalenceClasses, functionalDependencies);
+        Set<LogicalVariable> normSet = lgp.getColumnSet();
+        List<Mutable<ILogicalExpression>> newDistinctByList = new ArrayList<Mutable<ILogicalExpression>>();
+        for (Mutable<ILogicalExpression> p2Ref : expressions) {
+            ILogicalExpression p2 = p2Ref.getValue();
+            if (p2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                VariableReferenceExpression var2 = (VariableReferenceExpression) p2;
+                LogicalVariable v2 = var2.getVariableReference();
+                if (normSet.contains(v2)) {
+                    newDistinctByList.add(p2Ref);
+                }
+            } else {
+                newDistinctByList.add(p2Ref);
+            }
+        }
+        expressions.clear();
+        expressions.addAll(newDistinctByList);
+        return null;
+    }
 
-	@Override
-	public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		ctx.putEquivalenceClassMap(op, new HashMap<LogicalVariable, EquivalenceClass>());
-		ctx.putFDList(op, new ArrayList<FunctionalDependency>());
-		return null;
-	}
+    @Override
+    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        ctx.putEquivalenceClassMap(op, new HashMap<LogicalVariable, EquivalenceClass>());
+        ctx.putFDList(op, new ArrayList<FunctionalDependency>());
+        return null;
+    }
 
-	@Override
-	public Void visitExchangeOperator(ExchangeOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		propagateFDsAndEquivClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitExchangeOperator(ExchangeOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitGroupByOperator(GroupByOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
-		List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
-		ctx.putEquivalenceClassMap(op, equivalenceClasses);
-		ctx.putFDList(op, functionalDependencies);
+    @Override
+    public Void visitGroupByOperator(GroupByOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+        List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+        ctx.putEquivalenceClassMap(op, equivalenceClasses);
+        ctx.putFDList(op, functionalDependencies);
 
-		List<FunctionalDependency> inheritedFDs = new ArrayList<FunctionalDependency>();
-		for (ILogicalPlan p : op.getNestedPlans()) {
-			for (Mutable<ILogicalOperator> r : p.getRoots()) {
-				ILogicalOperator op2 = r.getValue();
-				equivalenceClasses.putAll(getOrComputeEqClasses(op2, ctx));
-				inheritedFDs.addAll(getOrComputeFDs(op2, ctx));
-			}
-		}
+        List<FunctionalDependency> inheritedFDs = new ArrayList<FunctionalDependency>();
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                ILogicalOperator op2 = r.getValue();
+                equivalenceClasses.putAll(getOrComputeEqClasses(op2, ctx));
+                inheritedFDs.addAll(getOrComputeFDs(op2, ctx));
+            }
+        }
 
-		ILogicalOperator op0 = op.getInputs().get(0).getValue();
-		inheritedFDs.addAll(getOrComputeFDs(op0, ctx));
-		Map<LogicalVariable, EquivalenceClass> inheritedEcs = getOrComputeEqClasses(op0, ctx);
-		for (FunctionalDependency inherited : inheritedFDs) {
-			boolean isCoveredByGbyOrDecorVars = true;
-			List<LogicalVariable> newHead = new ArrayList<LogicalVariable>(inherited.getHead().size());
-			for (LogicalVariable v : inherited.getHead()) {
-				LogicalVariable vnew = getNewGbyVar(op, v);
-				if (vnew == null) {
-					vnew = getNewDecorVar(op, v);
-					if (vnew == null) {
-						isCoveredByGbyOrDecorVars = false;
-					}
-					break;
-				}
-				newHead.add(vnew);
-			}
+        ILogicalOperator op0 = op.getInputs().get(0).getValue();
+        inheritedFDs.addAll(getOrComputeFDs(op0, ctx));
+        Map<LogicalVariable, EquivalenceClass> inheritedEcs = getOrComputeEqClasses(op0, ctx);
+        for (FunctionalDependency inherited : inheritedFDs) {
+            boolean isCoveredByGbyOrDecorVars = true;
+            List<LogicalVariable> newHead = new ArrayList<LogicalVariable>(inherited.getHead().size());
+            for (LogicalVariable v : inherited.getHead()) {
+                LogicalVariable vnew = getNewGbyVar(op, v);
+                if (vnew == null) {
+                    vnew = getNewDecorVar(op, v);
+                    if (vnew == null) {
+                        isCoveredByGbyOrDecorVars = false;
+                    }
+                    break;
+                }
+                newHead.add(vnew);
+            }
 
-			if (isCoveredByGbyOrDecorVars) {
-				List<LogicalVariable> newTail = new ArrayList<LogicalVariable>();
-				for (LogicalVariable v2 : inherited.getTail()) {
-					LogicalVariable v3 = getNewGbyVar(op, v2);
-					if (v3 != null) {
-						newTail.add(v3);
-					}
-				}
-				if (!newTail.isEmpty()) {
-					FunctionalDependency newFd = new FunctionalDependency(newHead, newTail);
-					functionalDependencies.add(newFd);
-				}
-			}
-		}
+            if (isCoveredByGbyOrDecorVars) {
+                List<LogicalVariable> newTail = new ArrayList<LogicalVariable>();
+                for (LogicalVariable v2 : inherited.getTail()) {
+                    LogicalVariable v3 = getNewGbyVar(op, v2);
+                    if (v3 != null) {
+                        newTail.add(v3);
+                    }
+                }
+                if (!newTail.isEmpty()) {
+                    FunctionalDependency newFd = new FunctionalDependency(newHead, newTail);
+                    functionalDependencies.add(newFd);
+                }
+            }
+        }
 
-		List<LogicalVariable> premiseGby = new LinkedList<LogicalVariable>();
-		List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gByList = op.getGroupByList();
-		for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gByList) {
-			premiseGby.add(p.first);
-		}
+        List<LogicalVariable> premiseGby = new LinkedList<LogicalVariable>();
+        List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gByList = op.getGroupByList();
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gByList) {
+            premiseGby.add(p.first);
+        }
 
-		List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> decorList = op.getDecorList();
+        List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> decorList = op.getDecorList();
 
-		LinkedList<LogicalVariable> conclDecor = new LinkedList<LogicalVariable>();
-		for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : decorList) {
-			conclDecor.add(GroupByOperator.getDecorVariable(p));
-		}
-		if (!conclDecor.isEmpty()) {
-			functionalDependencies.add(new FunctionalDependency(premiseGby, conclDecor));
-		}
+        LinkedList<LogicalVariable> conclDecor = new LinkedList<LogicalVariable>();
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : decorList) {
+            conclDecor.add(GroupByOperator.getDecorVariable(p));
+        }
+        if (!conclDecor.isEmpty()) {
+            functionalDependencies.add(new FunctionalDependency(premiseGby, conclDecor));
+        }
 
-		Set<LogicalVariable> gbySet = new HashSet<LogicalVariable>();
-		for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gByList) {
-			ILogicalExpression expr = p.second.getValue();
-			if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-				VariableReferenceExpression v = (VariableReferenceExpression) expr;
-				gbySet.add(v.getVariableReference());
-			}
-		}
-		LocalGroupingProperty lgp = new LocalGroupingProperty(gbySet);
-		lgp.normalizeGroupingColumns(inheritedEcs, inheritedFDs);
-		Set<LogicalVariable> normSet = lgp.getColumnSet();
-		List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> newGbyList = new ArrayList<Pair<LogicalVariable, Mutable<ILogicalExpression>>>();
-		boolean changed = false;
-		for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gByList) {
-			ILogicalExpression expr = p.second.getValue();
-			if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-				VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
-				LogicalVariable v2 = varRef.getVariableReference();
-				EquivalenceClass ec2 = inheritedEcs.get(v2);
-				LogicalVariable v3;
-				if (ec2 != null && !ec2.representativeIsConst()) {
-					v3 = ec2.getVariableRepresentative();
-				} else {
-					v3 = v2;
-				}
-				if (normSet.contains(v3)) {
-					newGbyList.add(p);
-				} else {
-					changed = true;
-					decorList.add(p);
-				}
-			} else {
-				newGbyList.add(p);
-			}
-		}
-		if (changed) {
-			AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Group-by list changed from "
-					+ GroupByOperator.veListToString(gByList) + " to " + GroupByOperator.veListToString(newGbyList)
-					+ ".\n");
-		}
-		gByList.clear();
-		gByList.addAll(newGbyList);
-		return null;
-	}
+        Set<LogicalVariable> gbySet = new HashSet<LogicalVariable>();
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gByList) {
+            ILogicalExpression expr = p.second.getValue();
+            if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                VariableReferenceExpression v = (VariableReferenceExpression) expr;
+                gbySet.add(v.getVariableReference());
+            }
+        }
+        LocalGroupingProperty lgp = new LocalGroupingProperty(gbySet);
+        lgp.normalizeGroupingColumns(inheritedEcs, inheritedFDs);
+        Set<LogicalVariable> normSet = lgp.getColumnSet();
+        List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> newGbyList = new ArrayList<Pair<LogicalVariable, Mutable<ILogicalExpression>>>();
+        boolean changed = false;
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gByList) {
+            ILogicalExpression expr = p.second.getValue();
+            if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+                LogicalVariable v2 = varRef.getVariableReference();
+                EquivalenceClass ec2 = inheritedEcs.get(v2);
+                LogicalVariable v3;
+                if (ec2 != null && !ec2.representativeIsConst()) {
+                    v3 = ec2.getVariableRepresentative();
+                } else {
+                    v3 = v2;
+                }
+                if (normSet.contains(v3)) {
+                    newGbyList.add(p);
+                } else {
+                    changed = true;
+                    decorList.add(p);
+                }
+            } else {
+                newGbyList.add(p);
+            }
+        }
+        if (changed) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Group-by list changed from "
+                    + GroupByOperator.veListToString(gByList) + " to " + GroupByOperator.veListToString(newGbyList)
+                    + ".\n");
+        }
+        gByList.clear();
+        gByList.addAll(newGbyList);
+        return null;
+    }
 
-	@Override
-	public Void visitInnerJoinOperator(InnerJoinOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
-		List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
-		ctx.putEquivalenceClassMap(op, equivalenceClasses);
-		ctx.putFDList(op, functionalDependencies);
-		ILogicalOperator op0 = op.getInputs().get(0).getValue();
-		ILogicalOperator op1 = op.getInputs().get(1).getValue();
-		functionalDependencies.addAll(getOrComputeFDs(op0, ctx));
-		functionalDependencies.addAll(getOrComputeFDs(op1, ctx));
-		equivalenceClasses.putAll(getOrComputeEqClasses(op0, ctx));
-		equivalenceClasses.putAll(getOrComputeEqClasses(op1, ctx));
-		ILogicalExpression expr = op.getCondition().getValue();
-		expr.getConstraintsAndEquivClasses(functionalDependencies, equivalenceClasses);
-		return null;
-	}
+    @Override
+    public Void visitInnerJoinOperator(InnerJoinOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+        List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+        ctx.putEquivalenceClassMap(op, equivalenceClasses);
+        ctx.putFDList(op, functionalDependencies);
+        ILogicalOperator op0 = op.getInputs().get(0).getValue();
+        ILogicalOperator op1 = op.getInputs().get(1).getValue();
+        functionalDependencies.addAll(getOrComputeFDs(op0, ctx));
+        functionalDependencies.addAll(getOrComputeFDs(op1, ctx));
+        equivalenceClasses.putAll(getOrComputeEqClasses(op0, ctx));
+        equivalenceClasses.putAll(getOrComputeEqClasses(op1, ctx));
+        ILogicalExpression expr = op.getCondition().getValue();
+        expr.getConstraintsAndEquivClasses(functionalDependencies, equivalenceClasses);
+        return null;
+    }
 
-	@Override
-	public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
-		List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
-		ctx.putEquivalenceClassMap(op, equivalenceClasses);
-		ctx.putFDList(op, functionalDependencies);
-		ILogicalOperator opLeft = op.getInputs().get(0).getValue();
-		ILogicalOperator opRight = op.getInputs().get(1).getValue();
-		functionalDependencies.addAll(getOrComputeFDs(opLeft, ctx));
-		functionalDependencies.addAll(getOrComputeFDs(opRight, ctx));
-		equivalenceClasses.putAll(getOrComputeEqClasses(opLeft, ctx));
-		equivalenceClasses.putAll(getOrComputeEqClasses(opRight, ctx));
+    @Override
+    public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+        List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+        ctx.putEquivalenceClassMap(op, equivalenceClasses);
+        ctx.putFDList(op, functionalDependencies);
+        ILogicalOperator opLeft = op.getInputs().get(0).getValue();
+        ILogicalOperator opRight = op.getInputs().get(1).getValue();
+        functionalDependencies.addAll(getOrComputeFDs(opLeft, ctx));
+        functionalDependencies.addAll(getOrComputeFDs(opRight, ctx));
+        equivalenceClasses.putAll(getOrComputeEqClasses(opLeft, ctx));
+        equivalenceClasses.putAll(getOrComputeEqClasses(opRight, ctx));
 
-		Collection<LogicalVariable> leftSideVars;
-		if (opLeft.getSchema() == null) {
-			leftSideVars = new LinkedList<LogicalVariable>();
-			VariableUtilities.getLiveVariables(opLeft, leftSideVars);
-			// actually, not all produced vars. are visible (due to projection)
-			// so using cached schema is better and faster
-		} else {
-			leftSideVars = opLeft.getSchema();
-		}
-		ILogicalExpression expr = op.getCondition().getValue();
-		expr.getConstraintsForOuterJoin(functionalDependencies, leftSideVars);
-		return null;
-	}
+        Collection<LogicalVariable> leftSideVars;
+        if (opLeft.getSchema() == null) {
+            leftSideVars = new LinkedList<LogicalVariable>();
+            VariableUtilities.getLiveVariables(opLeft, leftSideVars);
+            // actually, not all produced vars. are visible (due to projection)
+            // so using cached schema is better and faster
+        } else {
+            leftSideVars = opLeft.getSchema();
+        }
+        ILogicalExpression expr = op.getCondition().getValue();
+        expr.getConstraintsForOuterJoin(functionalDependencies, leftSideVars);
+        return null;
+    }
 
-	@Override
-	public Void visitLimitOperator(LimitOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		propagateFDsAndEquivClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitLimitOperator(LimitOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		AbstractLogicalOperator op1 = (AbstractLogicalOperator) op.getDataSourceReference().getValue();
-		ILogicalOperator inp1 = op1.getInputs().get(0).getValue();
-		Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
-		ctx.putEquivalenceClassMap(op, eqClasses);
-		List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>(getOrComputeFDs(inp1, ctx));
-		if (op1.getOperatorTag() == LogicalOperatorTag.GROUP) {
-			GroupByOperator gby = (GroupByOperator) op1;
-			LinkedList<LogicalVariable> tail = new LinkedList<LogicalVariable>();
-			for (LogicalVariable v : gby.getGbyVarList()) {
-				tail.add(v);
-				// all values for gby vars. are the same
-			}
-			FunctionalDependency gbyfd = new FunctionalDependency(new LinkedList<LogicalVariable>(), tail);
-			fds.add(gbyfd);
-		}
-		ctx.putFDList(op, fds);
-		return null;
-	}
+    @Override
+    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        AbstractLogicalOperator op1 = (AbstractLogicalOperator) op.getDataSourceReference().getValue();
+        ILogicalOperator inp1 = op1.getInputs().get(0).getValue();
+        Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+        ctx.putEquivalenceClassMap(op, eqClasses);
+        List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>(getOrComputeFDs(inp1, ctx));
+        if (op1.getOperatorTag() == LogicalOperatorTag.GROUP) {
+            GroupByOperator gby = (GroupByOperator) op1;
+            LinkedList<LogicalVariable> tail = new LinkedList<LogicalVariable>();
+            for (LogicalVariable v : gby.getGbyVarList()) {
+                tail.add(v);
+                // all values for gby vars. are the same
+            }
+            FunctionalDependency gbyfd = new FunctionalDependency(new LinkedList<LogicalVariable>(), tail);
+            fds.add(gbyfd);
+        }
+        ctx.putFDList(op, fds);
+        return null;
+    }
 
-	@Override
-	public Void visitOrderOperator(OrderOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		propagateFDsAndEquivClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitOrderOperator(OrderOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		throw new NotImplementedException();
-	}
+    @Override
+    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        throw new NotImplementedException();
+    }
 
-	@Override
-	public Void visitProjectOperator(ProjectOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		propagateFDsAndEquivClassesForUsedVars(op, ctx, op.getVariables());
-		return null;
-	}
+    @Override
+    public Void visitProjectOperator(ProjectOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClassesForUsedVars(op, ctx, op.getVariables());
+        return null;
+    }
 
-	@Override
-	public Void visitReplicateOperator(ReplicateOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		propagateFDsAndEquivClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitReplicateOperator(ReplicateOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitRunningAggregateOperator(RunningAggregateOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		propagateFDsAndEquivClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitRunningAggregateOperator(RunningAggregateOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitScriptOperator(ScriptOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		propagateFDsAndEquivClassesForUsedVars(op, ctx, op.getInputVariables());
-		return null;
-	}
+    @Override
+    public Void visitScriptOperator(ScriptOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClassesForUsedVars(op, ctx, op.getInputVariables());
+        return null;
+    }
 
-	@Override
-	public Void visitSelectOperator(SelectOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
-		List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
-		ctx.putEquivalenceClassMap(op, equivalenceClasses);
-		ctx.putFDList(op, functionalDependencies);
-		ILogicalOperator op0 = op.getInputs().get(0).getValue();
-		functionalDependencies.addAll(getOrComputeFDs(op0, ctx));
-		equivalenceClasses.putAll(getOrComputeEqClasses(op0, ctx));
-		ILogicalExpression expr = op.getCondition().getValue();
-		expr.getConstraintsAndEquivClasses(functionalDependencies, equivalenceClasses);
-		return null;
-	}
+    @Override
+    public Void visitSelectOperator(SelectOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+        List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+        ctx.putEquivalenceClassMap(op, equivalenceClasses);
+        ctx.putFDList(op, functionalDependencies);
+        ILogicalOperator op0 = op.getInputs().get(0).getValue();
+        functionalDependencies.addAll(getOrComputeFDs(op0, ctx));
+        equivalenceClasses.putAll(getOrComputeEqClasses(op0, ctx));
+        ILogicalExpression expr = op.getCondition().getValue();
+        expr.getConstraintsAndEquivClasses(functionalDependencies, equivalenceClasses);
+        return null;
+    }
 
-	@Override
-	public Void visitSubplanOperator(SubplanOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
-		List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
-		ctx.putEquivalenceClassMap(op, equivalenceClasses);
-		ctx.putFDList(op, functionalDependencies);
-		for (ILogicalPlan p : op.getNestedPlans()) {
-			for (Mutable<ILogicalOperator> r : p.getRoots()) {
-				ILogicalOperator op2 = r.getValue();
-				equivalenceClasses.putAll(getOrComputeEqClasses(op2, ctx));
-				functionalDependencies.addAll(getOrComputeFDs(op2, ctx));
-			}
-		}
-		return null;
-	}
+    @Override
+    public Void visitSubplanOperator(SubplanOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        Map<LogicalVariable, EquivalenceClass> equivalenceClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+        List<FunctionalDependency> functionalDependencies = new ArrayList<FunctionalDependency>();
+        ctx.putEquivalenceClassMap(op, equivalenceClasses);
+        ctx.putFDList(op, functionalDependencies);
+        for (ILogicalPlan p : op.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                ILogicalOperator op2 = r.getValue();
+                equivalenceClasses.putAll(getOrComputeEqClasses(op2, ctx));
+                functionalDependencies.addAll(getOrComputeFDs(op2, ctx));
+            }
+        }
+        return null;
+    }
 
-	@Override
-	public Void visitUnionOperator(UnionAllOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		setEmptyFDsEqClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitUnionOperator(UnionAllOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        setEmptyFDsEqClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitUnnestMapOperator(UnnestMapOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		fdsEqClassesForAbstractUnnestOperator(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitUnnestMapOperator(UnnestMapOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        fdsEqClassesForAbstractUnnestOperator(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitExternalDataAccessByRIDOperator(ExternalDataAccessByRIDOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		fdsEqClassesForExternalAccessOperator(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitUnnestOperator(UnnestOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        fdsEqClassesForAbstractUnnestOperator(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitUnnestOperator(UnnestOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		fdsEqClassesForAbstractUnnestOperator(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitWriteOperator(WriteOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        // propagateFDsAndEquivClasses(op, ctx);
+        setEmptyFDsEqClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitWriteOperator(WriteOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		// propagateFDsAndEquivClasses(op, ctx);
-		setEmptyFDsEqClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitDistributeResultOperator(DistributeResultOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        // propagateFDsAndEquivClasses(op, ctx);
+        setEmptyFDsEqClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitDistributeResultOperator(DistributeResultOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		// propagateFDsAndEquivClasses(op, ctx);
-		setEmptyFDsEqClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitWriteResultOperator(WriteResultOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        // propagateFDsAndEquivClasses(op, ctx);
+        setEmptyFDsEqClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitWriteResultOperator(WriteResultOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		// propagateFDsAndEquivClasses(op, ctx);
-		setEmptyFDsEqClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitInsertDeleteOperator(InsertDeleteOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        setEmptyFDsEqClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitInsertDeleteOperator(InsertDeleteOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		setEmptyFDsEqClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        setEmptyFDsEqClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		setEmptyFDsEqClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitSinkOperator(SinkOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        setEmptyFDsEqClasses(op, ctx);
+        return null;
+    }
 
-	@Override
-	public Void visitSinkOperator(SinkOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		setEmptyFDsEqClasses(op, ctx);
-		return null;
-	}
+    private void propagateFDsAndEquivClasses(ILogicalOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        ILogicalOperator inp1 = op.getInputs().get(0).getValue();
+        Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+        ctx.putEquivalenceClassMap(op, eqClasses);
+        List<FunctionalDependency> fds = getOrComputeFDs(inp1, ctx);
+        ctx.putFDList(op, fds);
+    }
 
-	private void propagateFDsAndEquivClasses(ILogicalOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		ILogicalOperator inp1 = op.getInputs().get(0).getValue();
-		Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
-		ctx.putEquivalenceClassMap(op, eqClasses);
-		List<FunctionalDependency> fds = getOrComputeFDs(inp1, ctx);
-		ctx.putFDList(op, fds);
-	}
+    private Map<LogicalVariable, EquivalenceClass> getOrComputeEqClasses(ILogicalOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        Map<LogicalVariable, EquivalenceClass> eqClasses = ctx.getEquivalenceClassMap(op);
+        if (eqClasses == null) {
+            op.accept(this, ctx);
+            eqClasses = ctx.getEquivalenceClassMap(op);
+        }
+        return eqClasses;
+    }
 
-	private Map<LogicalVariable, EquivalenceClass> getOrComputeEqClasses(ILogicalOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		Map<LogicalVariable, EquivalenceClass> eqClasses = ctx.getEquivalenceClassMap(op);
-		if (eqClasses == null) {
-			op.accept(this, ctx);
-			eqClasses = ctx.getEquivalenceClassMap(op);
-		}
-		return eqClasses;
-	}
+    private List<FunctionalDependency> getOrComputeFDs(ILogicalOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        List<FunctionalDependency> fds = ctx.getFDList(op);
+        if (fds == null) {
+            op.accept(this, ctx);
+            fds = ctx.getFDList(op);
+        }
+        return fds;
+    }
 
-	private List<FunctionalDependency> getOrComputeFDs(ILogicalOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		List<FunctionalDependency> fds = ctx.getFDList(op);
-		if (fds == null) {
-			op.accept(this, ctx);
-			fds = ctx.getFDList(op);
-		}
-		return fds;
-	}
+    private void propagateFDsAndEquivClassesForUsedVars(ILogicalOperator op, IOptimizationContext ctx,
+            List<LogicalVariable> usedVariables) throws AlgebricksException {
+        ILogicalOperator op2 = op.getInputs().get(0).getValue();
+        Map<LogicalVariable, EquivalenceClass> eqClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+        ctx.putEquivalenceClassMap(op, eqClasses);
+        List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>();
+        ctx.putFDList(op, fds);
 
-	private void propagateFDsAndEquivClassesForUsedVars(ILogicalOperator op, IOptimizationContext ctx,
-			List<LogicalVariable> usedVariables) throws AlgebricksException {
-		ILogicalOperator op2 = op.getInputs().get(0).getValue();
-		Map<LogicalVariable, EquivalenceClass> eqClasses = new HashMap<LogicalVariable, EquivalenceClass>();
-		ctx.putEquivalenceClassMap(op, eqClasses);
-		List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>();
-		ctx.putFDList(op, fds);
+        Map<LogicalVariable, EquivalenceClass> chldClasses = getOrComputeEqClasses(op2, ctx);
+        for (LogicalVariable v : usedVariables) {
+            EquivalenceClass ec = eqClasses.get(v);
+            if (ec == null) {
+                EquivalenceClass oc = chldClasses.get(v);
+                if (oc == null) {
+                    continue;
+                }
+                List<LogicalVariable> m = new LinkedList<LogicalVariable>();
+                for (LogicalVariable v2 : oc.getMembers()) {
+                    if (usedVariables.contains(v2)) {
+                        m.add(v2);
+                    }
+                }
+                EquivalenceClass nc;
+                if (oc.representativeIsConst()) {
+                    nc = new EquivalenceClass(m, oc.getConstRepresentative());
+                } else if (m.contains(oc.getVariableRepresentative())) {
+                    nc = new EquivalenceClass(m, oc.getVariableRepresentative());
+                } else {
+                    nc = new EquivalenceClass(m, v);
+                }
+                for (LogicalVariable v3 : m) {
+                    eqClasses.put(v3, nc);
+                }
+            }
+        }
 
-		Map<LogicalVariable, EquivalenceClass> chldClasses = getOrComputeEqClasses(op2, ctx);
-		for (LogicalVariable v : usedVariables) {
-			EquivalenceClass ec = eqClasses.get(v);
-			if (ec == null) {
-				EquivalenceClass oc = chldClasses.get(v);
-				if (oc == null) {
-					continue;
-				}
-				List<LogicalVariable> m = new LinkedList<LogicalVariable>();
-				for (LogicalVariable v2 : oc.getMembers()) {
-					if (usedVariables.contains(v2)) {
-						m.add(v2);
-					}
-				}
-				EquivalenceClass nc;
-				if (oc.representativeIsConst()) {
-					nc = new EquivalenceClass(m, oc.getConstRepresentative());
-				} else if (m.contains(oc.getVariableRepresentative())) {
-					nc = new EquivalenceClass(m, oc.getVariableRepresentative());
-				} else {
-					nc = new EquivalenceClass(m, v);
-				}
-				for (LogicalVariable v3 : m) {
-					eqClasses.put(v3, nc);
-				}
-			}
-		}
+        List<FunctionalDependency> chldFds = getOrComputeFDs(op2, ctx);
+        for (FunctionalDependency fd : chldFds) {
+            if (!usedVariables.containsAll(fd.getHead())) {
+                continue;
+            }
+            List<LogicalVariable> tl = new LinkedList<LogicalVariable>();
+            for (LogicalVariable v : fd.getTail()) {
+                if (usedVariables.contains(v)) {
+                    tl.add(v);
+                }
+            }
+            if (!tl.isEmpty()) {
+                FunctionalDependency newFd = new FunctionalDependency(fd.getHead(), tl);
+                fds.add(newFd);
+            }
+        }
+    }
 
-		List<FunctionalDependency> chldFds = getOrComputeFDs(op2, ctx);
-		for (FunctionalDependency fd : chldFds) {
-			if (!usedVariables.containsAll(fd.getHead())) {
-				continue;
-			}
-			List<LogicalVariable> tl = new LinkedList<LogicalVariable>();
-			for (LogicalVariable v : fd.getTail()) {
-				if (usedVariables.contains(v)) {
-					tl.add(v);
-				}
-			}
-			if (!tl.isEmpty()) {
-				FunctionalDependency newFd = new FunctionalDependency(fd.getHead(), tl);
-				fds.add(newFd);
-			}
-		}
-	}
+    private void fdsEqClassesForAbstractUnnestOperator(AbstractUnnestOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        ILogicalOperator inp1 = op.getInputs().get(0).getValue();
+        Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
+        ctx.putEquivalenceClassMap(op, eqClasses);
+        List<FunctionalDependency> fds = getOrComputeFDs(inp1, ctx);
+        ctx.putFDList(op, fds);
 
-	private void fdsEqClassesForAbstractUnnestOperator(AbstractUnnestOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		ILogicalOperator inp1 = op.getInputs().get(0).getValue();
-		Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
-		ctx.putEquivalenceClassMap(op, eqClasses);
-		List<FunctionalDependency> fds = getOrComputeFDs(inp1, ctx);
-		ctx.putFDList(op, fds);
+        ILogicalExpression expr = op.getExpressionRef().getValue();
+        if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+            AbstractFunctionCallExpression afe = (AbstractFunctionCallExpression) expr;
+            if (afe.getKind() == FunctionKind.UNNEST && ((UnnestingFunctionCallExpression) afe).returnsUniqueValues()) {
+                List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+                VariableUtilities.getLiveVariables(op, vars);
+                ArrayList<LogicalVariable> h = new ArrayList<LogicalVariable>();
+                h.addAll(op.getVariables());
+                FunctionalDependency fd = new FunctionalDependency(h, vars);
+                fds.add(fd);
+            }
+        }
+    }
 
-		ILogicalExpression expr = op.getExpressionRef().getValue();
-		if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-			AbstractFunctionCallExpression afe = (AbstractFunctionCallExpression) expr;
-			if (afe.getKind() == FunctionKind.UNNEST && ((UnnestingFunctionCallExpression) afe).returnsUniqueValues()) {
-				List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
-				VariableUtilities.getLiveVariables(op, vars);
-				ArrayList<LogicalVariable> h = new ArrayList<LogicalVariable>();
-				h.addAll(op.getVariables());
-				FunctionalDependency fd = new FunctionalDependency(h, vars);
-				fds.add(fd);
-			}
-		}
-	}
+    public static void setEmptyFDsEqClasses(ILogicalOperator op, IOptimizationContext ctx) {
+        Map<LogicalVariable, EquivalenceClass> eqClasses = new HashMap<LogicalVariable, EquivalenceClass>();
+        ctx.putEquivalenceClassMap(op, eqClasses);
+        List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>();
+        ctx.putFDList(op, fds);
+    }
 
-	private void fdsEqClassesForExternalAccessOperator(ExternalDataAccessByRIDOperator op, IOptimizationContext ctx)
-			throws AlgebricksException {
-		ILogicalOperator inp1 = op.getInputs().get(0).getValue();
-		Map<LogicalVariable, EquivalenceClass> eqClasses = getOrComputeEqClasses(inp1, ctx);
-		ctx.putEquivalenceClassMap(op, eqClasses);
-		List<FunctionalDependency> fds = getOrComputeFDs(inp1, ctx);
-		ctx.putFDList(op, fds);
-		List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
-		VariableUtilities.getLiveVariables(op, vars);
-		ArrayList<LogicalVariable> h = new ArrayList<LogicalVariable>();
-		h.addAll(op.getVariables());
-		FunctionalDependency fd = new FunctionalDependency(h, vars);
-		fds.add(fd);
-	}
+    private LogicalVariable getNewGbyVar(GroupByOperator g, LogicalVariable v) {
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : g.getGroupByList()) {
+            ILogicalExpression e = p.second.getValue();
+            if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                LogicalVariable v2 = ((VariableReferenceExpression) e).getVariableReference();
+                if (v2 == v) {
+                    return p.first;
+                }
+            }
+        }
+        return null;
+    }
 
-	public static void setEmptyFDsEqClasses(ILogicalOperator op, IOptimizationContext ctx) {
-		Map<LogicalVariable, EquivalenceClass> eqClasses = new HashMap<LogicalVariable, EquivalenceClass>();
-		ctx.putEquivalenceClassMap(op, eqClasses);
-		List<FunctionalDependency> fds = new ArrayList<FunctionalDependency>();
-		ctx.putFDList(op, fds);
-	}
+    private LogicalVariable getNewDecorVar(GroupByOperator g, LogicalVariable v) {
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : g.getDecorList()) {
+            ILogicalExpression e = p.second.getValue();
+            if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                LogicalVariable v2 = ((VariableReferenceExpression) e).getVariableReference();
+                if (v2 == v) {
+                    return (p.first != null) ? p.first : v2;
+                }
+            }
+        }
+        return null;
+    }
 
-	private LogicalVariable getNewGbyVar(GroupByOperator g, LogicalVariable v) {
-		for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : g.getGroupByList()) {
-			ILogicalExpression e = p.second.getValue();
-			if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-				LogicalVariable v2 = ((VariableReferenceExpression) e).getVariableReference();
-				if (v2 == v) {
-					return p.first;
-				}
-			}
-		}
-		return null;
-	}
-
-	private LogicalVariable getNewDecorVar(GroupByOperator g, LogicalVariable v) {
-		for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : g.getDecorList()) {
-			ILogicalExpression e = p.second.getValue();
-			if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-				LogicalVariable v2 = ((VariableReferenceExpression) e).getVariableReference();
-				if (v2 == v) {
-					return (p.first != null) ? p.first : v2;
-				}
-			}
-		}
-		return null;
-	}
-
-	@Override
-	public Void visitExtensionOperator(ExtensionOperator op, IOptimizationContext ctx) throws AlgebricksException {
-		propagateFDsAndEquivClasses(op, ctx);
-		return null;
-	}
+    @Override
+    public Void visitExtensionOperator(ExtensionOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
 
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 0561091..97d257a 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -41,7 +41,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -343,14 +342,6 @@
         isomorphic = op.getExpressionRef().getValue().equals(unnestOpArg.getExpressionRef().getValue());
         return isomorphic;
     }
-    
-    //not sure if this is correct!
-    @Override
-	public Boolean visitExternalDataAccessByRIDOperator(
-			ExternalDataAccessByRIDOperator op, ILogicalOperator arg)
-			throws AlgebricksException {
-    	return Boolean.FALSE;
-	}
 
     @Override
     public Boolean visitDataScanOperator(DataSourceScanOperator op, ILogicalOperator arg) throws AlgebricksException {
@@ -739,16 +730,6 @@
             return new UnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
                     new ArrayList<Object>(op.getVariableTypes()), op.propagatesInput());
         }
-        
-        @Override
-		public ILogicalOperator visitExternalDataAccessByRIDOperator(
-				ExternalDataAccessByRIDOperator op, Void arg)
-        {
-        ArrayList<LogicalVariable> newInputList = new ArrayList<LogicalVariable>();
-        newInputList.addAll(op.getVariables());
-        return new ExternalDataAccessByRIDOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
-                new ArrayList<Object>(op.getVariableTypes()));
-		}
 
         @Override
         public ILogicalOperator visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 407f7dc8..eecace7 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -40,7 +40,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -199,13 +198,6 @@
         mapVariablesStandard(op, arg);
         return null;
     }
-    
-    @Override
-	public Void visitExternalDataAccessByRIDOperator(
-			ExternalDataAccessByRIDOperator op, ILogicalOperator arg)
-			throws AlgebricksException {
-		return null;
-	}
 
     @Override
     public Void visitDataScanOperator(DataSourceScanOperator op, ILogicalOperator arg) throws AlgebricksException {
@@ -430,5 +422,4 @@
         return null;
     }
 
-
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index c978bac..277b5a3 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -32,7 +32,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -326,12 +325,4 @@
         return null;
     }
 
-	@Override
-	public Void visitExternalDataAccessByRIDOperator(
-			ExternalDataAccessByRIDOperator op, IOptimizationContext arg)
-			throws AlgebricksException {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index b76078e..8673a77 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -35,7 +35,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

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

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

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

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

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

@@ -200,12 +199,6 @@
     public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {

         producedVariables.addAll(op.getVariables());

         return null;

-    }
-    
-    @Override
-    public Void visitExternalDataAccessByRIDOperator(ExternalDataAccessByRIDOperator op, Void arg) throws AlgebricksException {
-        producedVariables.add(op.getVariables().get(0));
-        return null;
     }

 

     @Override

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 82ea1a7..5606e72 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -36,7 +36,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

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

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

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

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

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

@@ -227,13 +226,7 @@
         standardLayout(op);

         return null;

     }

-
-    @Override
-    public Void visitExternalDataAccessByRIDOperator(ExternalDataAccessByRIDOperator op, Void arg) throws AlgebricksException {
-    	VariableUtilities.getProducedVariables(op, schemaVariables);
-        return null;
-    }
-    

+

     @Override

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

         standardLayout(op);

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 3964e56..3d6b989 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -36,7 +36,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

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

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

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

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

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

@@ -300,21 +299,6 @@
         op.getExpressionRef().getValue().substituteVar(pair.first, pair.second);

         substVarTypes(op, pair);

         return null;

-    }
-    
-    @Override
-    public Void visitExternalDataAccessByRIDOperator(ExternalDataAccessByRIDOperator op, Pair<LogicalVariable, LogicalVariable> pair)
-            throws AlgebricksException {
-        List<LogicalVariable> variables = op.getVariables();
-        for (int i = 0; i < variables.size(); i++) {
-            if (variables.get(i) == pair.first) {
-                variables.set(i, pair.second);
-                return null;
-            }
-        }
-        op.getExpressionRef().getValue().substituteVar(pair.first, pair.second);
-        substVarTypes(op, pair);
-        return null;
     }

 

     @Override

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 9b4164a..d05adfb 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -35,7 +35,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;

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

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

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

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

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

@@ -280,13 +279,7 @@
         op.getExpressionRef().getValue().getUsedVariables(usedVariables);

         return null;

     }

-
-    @Override
-    public Void visitExternalDataAccessByRIDOperator(ExternalDataAccessByRIDOperator op, Void arg) {
-        op.getExpressionRef().getValue().getUsedVariables(usedVariables);
-        return null;
-    }
-    

+

     @Override

     public Void visitUnnestOperator(UnnestOperator op, Void arg) {

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

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index a20d20d..49ec269 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -33,7 +33,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -235,13 +234,6 @@
         addIndent(buffer, indent).append("unnest-map " + op.getVariables() + " <- " + op.getExpressionRef().getValue());
         return buffer.toString();
     }
-    
-    @Override
-    public String visitExternalDataAccessByRIDOperator(ExternalDataAccessByRIDOperator op, Integer indent) throws AlgebricksException {
-    	StringBuilder buffer = new StringBuilder();
-        addIndent(buffer, indent).append("external-access-by-rid " + op.getVariables() + " <- " + op.getExpressionRef().getValue());
-        return buffer.toString();
-    }
 
     @Override
     public String visitDataScanOperator(DataSourceScanOperator op, Integer indent) {
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index ca7d21b..0467d63 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -23,7 +23,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -89,8 +88,6 @@
     public R visitUnnestOperator(UnnestOperator op, T arg) throws AlgebricksException;
 
     public R visitUnnestMapOperator(UnnestMapOperator op, T arg) throws AlgebricksException;
-    
-    public R visitExternalDataAccessByRIDOperator(ExternalDataAccessByRIDOperator op, T arg) throws AlgebricksException;
 
     public R visitDataScanOperator(DataSourceScanOperator op, T arg) throws AlgebricksException;
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
index 56200e4..4638118 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
@@ -111,6 +111,11 @@
             BitSet sourceBitmap);
 
     /**
+     * Indicate whether the connector is an all-producers-to-all-consumers connector
+     */
+    public boolean allProducersToAllConsumers();
+
+    /**
      * Gets the display name.
      */
     public String getDisplayName();
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksDataException.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksDataException.java
index 6390abf..aab59c8 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksDataException.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/exceptions/HyracksDataException.java
@@ -17,6 +17,8 @@
 public class HyracksDataException extends HyracksException {
     private static final long serialVersionUID = 1L;
 
+    private String nodeId;
+
     public HyracksDataException() {
     }
 
@@ -24,11 +26,19 @@
         super(message);
     }
 
+    public HyracksDataException(Throwable cause) {
+        super(cause);
+    }
+
     public HyracksDataException(String message, Throwable cause) {
         super(message, cause);
     }
 
-    public HyracksDataException(Throwable cause) {
-        super(cause);
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    public String getNodeId() {
+        return nodeId;
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index 67ba2b6..6785d6f 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -189,36 +189,37 @@
 
     public void notifyHeartbeat(HeartbeatData hbData) {
         lastHeartbeatDuration = 0;
-
         hbTime[rrdPtr] = System.currentTimeMillis();
-        heapInitSize[rrdPtr] = hbData.heapInitSize;
-        heapUsedSize[rrdPtr] = hbData.heapUsedSize;
-        heapCommittedSize[rrdPtr] = hbData.heapCommittedSize;
-        heapMaxSize[rrdPtr] = hbData.heapMaxSize;
-        nonheapInitSize[rrdPtr] = hbData.nonheapInitSize;
-        nonheapUsedSize[rrdPtr] = hbData.nonheapUsedSize;
-        nonheapCommittedSize[rrdPtr] = hbData.nonheapCommittedSize;
-        nonheapMaxSize[rrdPtr] = hbData.nonheapMaxSize;
-        threadCount[rrdPtr] = hbData.threadCount;
-        peakThreadCount[rrdPtr] = hbData.peakThreadCount;
-        systemLoadAverage[rrdPtr] = hbData.systemLoadAverage;
-        int gcN = hbSchema.getGarbageCollectorInfos().length;
-        for (int i = 0; i < gcN; ++i) {
-            gcCollectionCounts[i][rrdPtr] = hbData.gcCollectionCounts[i];
-            gcCollectionTimes[i][rrdPtr] = hbData.gcCollectionTimes[i];
+        if (hbData != null) {
+            heapInitSize[rrdPtr] = hbData.heapInitSize;
+            heapUsedSize[rrdPtr] = hbData.heapUsedSize;
+            heapCommittedSize[rrdPtr] = hbData.heapCommittedSize;
+            heapMaxSize[rrdPtr] = hbData.heapMaxSize;
+            nonheapInitSize[rrdPtr] = hbData.nonheapInitSize;
+            nonheapUsedSize[rrdPtr] = hbData.nonheapUsedSize;
+            nonheapCommittedSize[rrdPtr] = hbData.nonheapCommittedSize;
+            nonheapMaxSize[rrdPtr] = hbData.nonheapMaxSize;
+            threadCount[rrdPtr] = hbData.threadCount;
+            peakThreadCount[rrdPtr] = hbData.peakThreadCount;
+            systemLoadAverage[rrdPtr] = hbData.systemLoadAverage;
+            int gcN = hbSchema.getGarbageCollectorInfos().length;
+            for (int i = 0; i < gcN; ++i) {
+                gcCollectionCounts[i][rrdPtr] = hbData.gcCollectionCounts[i];
+                gcCollectionTimes[i][rrdPtr] = hbData.gcCollectionTimes[i];
+            }
+            netPayloadBytesRead[rrdPtr] = hbData.netPayloadBytesRead;
+            netPayloadBytesWritten[rrdPtr] = hbData.netPayloadBytesWritten;
+            netSignalingBytesRead[rrdPtr] = hbData.netSignalingBytesRead;
+            netSignalingBytesWritten[rrdPtr] = hbData.netSignalingBytesWritten;
+            datasetNetPayloadBytesRead[rrdPtr] = hbData.datasetNetPayloadBytesRead;
+            datasetNetPayloadBytesWritten[rrdPtr] = hbData.datasetNetPayloadBytesWritten;
+            datasetNetSignalingBytesRead[rrdPtr] = hbData.datasetNetSignalingBytesRead;
+            datasetNetSignalingBytesWritten[rrdPtr] = hbData.datasetNetSignalingBytesWritten;
+            ipcMessagesSent[rrdPtr] = hbData.ipcMessagesSent;
+            ipcMessageBytesSent[rrdPtr] = hbData.ipcMessageBytesSent;
+            ipcMessagesReceived[rrdPtr] = hbData.ipcMessagesReceived;
+            ipcMessageBytesReceived[rrdPtr] = hbData.ipcMessageBytesReceived;
         }
-        netPayloadBytesRead[rrdPtr] = hbData.netPayloadBytesRead;
-        netPayloadBytesWritten[rrdPtr] = hbData.netPayloadBytesWritten;
-        netSignalingBytesRead[rrdPtr] = hbData.netSignalingBytesRead;
-        netSignalingBytesWritten[rrdPtr] = hbData.netSignalingBytesWritten;
-        datasetNetPayloadBytesRead[rrdPtr] = hbData.datasetNetPayloadBytesRead;
-        datasetNetPayloadBytesWritten[rrdPtr] = hbData.datasetNetPayloadBytesWritten;
-        datasetNetSignalingBytesRead[rrdPtr] = hbData.datasetNetSignalingBytesRead;
-        datasetNetSignalingBytesWritten[rrdPtr] = hbData.datasetNetSignalingBytesWritten;
-        ipcMessagesSent[rrdPtr] = hbData.ipcMessagesSent;
-        ipcMessageBytesSent[rrdPtr] = hbData.ipcMessageBytesSent;
-        ipcMessagesReceived[rrdPtr] = hbData.ipcMessagesReceived;
-        ipcMessageBytesReceived[rrdPtr] = hbData.ipcMessageBytesReceived;
         rrdPtr = (rrdPtr + 1) % RRD_SIZE;
     }
 
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 2166620..bae0eb5 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
@@ -46,6 +46,7 @@
 import edu.uci.ics.hyracks.control.cc.scheduler.ActivityPartitionDetails;
 import edu.uci.ics.hyracks.control.cc.scheduler.JobScheduler;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.utils.ExceptionUtils;
 
 public class JobRun implements IJobStatusConditionVariable {
     private final DeploymentId deploymentId;
@@ -347,7 +348,7 @@
                                 taskAttempt.put("end-time", ta.getEndTime());
                                 List<Exception> exceptions = ta.getExceptions();
                                 if (exceptions != null && !exceptions.isEmpty()) {
-                                    List<Exception> filteredExceptions = ExceptionFilterUtils
+                                    List<Exception> filteredExceptions = ExceptionUtils
                                             .getActualExceptions(exceptions);
                                     for (Exception exception : filteredExceptions) {
                                         StringWriter exceptionWriter = new StringWriter();
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
index 4d2ad6b..3863eda 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
@@ -190,18 +190,29 @@
                     ActivityId ac2 = ac.getConsumerActivity(cdId);
                     Task[] ac2TaskStates = activityPlanMap.get(ac2).getTasks();
                     int nConsumers = ac2TaskStates.length;
-                    for (int i = 0; i < nProducers; ++i) {
-                        c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
-                        List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(ac1TaskStates[i]
-                                .getTaskId());
-                        if (cInfoList == null) {
-                            cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
-                            taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
-                        }
-                        for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
+                    if (c.allProducersToAllConsumers()) {
+                        List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
+                        for (int j = 0; j < nConsumers; j++) {
                             TaskId targetTID = ac2TaskStates[j].getTaskId();
                             cInfoList.add(Pair.<TaskId, ConnectorDescriptorId> of(targetTID, cdId));
                         }
+                        for (int i = 0; i < nProducers; ++i) {
+                            taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
+                        }
+                    } else {
+                        for (int i = 0; i < nProducers; ++i) {
+                            c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+                            List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(ac1TaskStates[i]
+                                    .getTaskId());
+                            if (cInfoList == null) {
+                                cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
+                                taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
+                            }
+                            for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
+                                TaskId targetTID = ac2TaskStates[j].getTaskId();
+                                cInfoList.add(Pair.<TaskId, ConnectorDescriptorId> of(targetTID, cdId));
+                            }
+                        }
                     }
                 }
             }
@@ -341,9 +352,15 @@
                     int nConsumers = ac2TaskStates.length;
 
                     int[] fanouts = new int[nProducers];
-                    for (int i = 0; i < nProducers; ++i) {
-                        c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
-                        fanouts[i] = targetBitmap.cardinality();
+                    if (c.allProducersToAllConsumers()) {
+                        for (int i = 0; i < nProducers; ++i) {
+                            fanouts[i] = nConsumers;
+                        }
+                    } else {
+                        for (int i = 0; i < nProducers; ++i) {
+                            c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+                            fanouts[i] = targetBitmap.cardinality();
+                        }
                     }
                     IConnectorPolicy cp = assignConnectorPolicy(ac, c, nProducers, nConsumers, fanouts);
                     cPolicyMap.put(cdId, cp);
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 0b8346b..fd6360a 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
@@ -27,6 +27,9 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.json.JSONException;
+import org.json.JSONObject;
+
 import edu.uci.ics.hyracks.api.constraints.Constraint;
 import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
 import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
@@ -45,6 +48,7 @@
 import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
 import edu.uci.ics.hyracks.control.cc.job.ActivityClusterPlan;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
 import edu.uci.ics.hyracks.control.cc.job.Task;
@@ -458,13 +462,14 @@
     private void abortJob(List<Exception> exceptions) {
         Set<TaskCluster> inProgressTaskClustersCopy = new HashSet<TaskCluster>(inProgressTaskClusters);
         for (TaskCluster tc : inProgressTaskClustersCopy) {
-            abortTaskCluster(findLastTaskClusterAttempt(tc));
+            abortTaskCluster(findLastTaskClusterAttempt(tc), TaskClusterAttempt.TaskClusterStatus.ABORTED);
         }
         assert inProgressTaskClusters.isEmpty();
         ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.FAILURE, exceptions));
     }
 
-    private void abortTaskCluster(TaskClusterAttempt tcAttempt) {
+    private void abortTaskCluster(TaskClusterAttempt tcAttempt,
+            TaskClusterAttempt.TaskClusterStatus failedOrAbortedStatus) {
         LOGGER.fine("Aborting task cluster: " + tcAttempt.getAttempt());
         Set<TaskAttemptId> abortTaskIds = new HashSet<TaskAttemptId>();
         Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<String, List<TaskAttemptId>>();
@@ -477,11 +482,13 @@
                 ta.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
                 ta.setEndTime(System.currentTimeMillis());
                 List<TaskAttemptId> abortTaskAttempts = abortTaskAttemptMap.get(ta.getNodeId());
-                if (abortTaskAttempts == null) {
+                if (status == TaskAttempt.TaskStatus.RUNNING && abortTaskAttempts == null) {
                     abortTaskAttempts = new ArrayList<TaskAttemptId>();
                     abortTaskAttemptMap.put(ta.getNodeId(), abortTaskAttempts);
                 }
-                abortTaskAttempts.add(taId);
+                if (status == TaskAttempt.TaskStatus.RUNNING) {
+                    abortTaskAttempts.add(taId);
+                }
             }
         }
         final JobId jobId = jobRun.getJobId();
@@ -505,6 +512,9 @@
         PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
         pmm.removeUncommittedPartitions(tc.getProducedPartitions(), abortTaskIds);
         pmm.removePartitionRequests(tc.getRequiredPartitions(), abortTaskIds);
+
+        tcAttempt.setStatus(failedOrAbortedStatus);
+        tcAttempt.setEndTime(System.currentTimeMillis());
     }
 
     private void abortDoomedTaskClusters() throws HyracksException {
@@ -519,9 +529,7 @@
         for (TaskCluster tc : doomedTaskClusters) {
             TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
             if (tca != null) {
-                abortTaskCluster(tca);
-                tca.setEndTime(System.currentTimeMillis());
-                tca.setStatus(TaskClusterAttempt.TaskClusterStatus.ABORTED);
+                abortTaskCluster(tca, TaskClusterAttempt.TaskClusterStatus.ABORTED);
             }
         }
     }
@@ -608,9 +616,7 @@
             if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
                 LOGGER.fine("Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
                 ta.setStatus(TaskAttempt.TaskStatus.FAILED, exceptions);
-                abortTaskCluster(lastAttempt);
-                lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
-                lastAttempt.setEndTime(System.currentTimeMillis());
+                abortTaskCluster(lastAttempt, TaskClusterAttempt.TaskClusterStatus.FAILED);
                 abortDoomedTaskClusters();
                 if (lastAttempt.getAttempt() >= jobRun.getActivityClusterGraph().getMaxReattempts()) {
                     abortJob(exceptions);
@@ -635,32 +641,41 @@
     public void notifyNodeFailures(Set<String> deadNodes) {
         try {
             jobRun.getPartitionMatchMaker().notifyNodeFailures(deadNodes);
+            jobRun.getParticipatingNodeIds().removeAll(deadNodes);
+            jobRun.getCleanupPendingNodeIds().removeAll(deadNodes);
+            if (jobRun.getPendingStatus() != null && jobRun.getCleanupPendingNodeIds().isEmpty()) {
+                finishJob(jobRun);
+                return;
+            }
             for (ActivityCluster ac : jobRun.getActivityClusterGraph().getActivityClusterMap().values()) {
-                TaskCluster[] taskClusters = getActivityClusterPlan(ac).getTaskClusters();
-                if (taskClusters != null) {
-                    for (TaskCluster tc : taskClusters) {
-                        TaskClusterAttempt lastTaskClusterAttempt = findLastTaskClusterAttempt(tc);
-                        if (lastTaskClusterAttempt != null
-                                && (lastTaskClusterAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED || lastTaskClusterAttempt
-                                        .getStatus() == TaskClusterAttempt.TaskClusterStatus.RUNNING)) {
-                            boolean abort = false;
-                            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,
-                                            Collections.singletonList(new Exception("Node " + ta.getNodeId()
-                                                    + " failed")));
-                                    ta.setEndTime(System.currentTimeMillis());
-                                    abort = true;
+                if (isPlanned(ac)) {
+                    TaskCluster[] taskClusters = getActivityClusterPlan(ac).getTaskClusters();
+                    if (taskClusters != null) {
+                        for (TaskCluster tc : taskClusters) {
+                            TaskClusterAttempt lastTaskClusterAttempt = findLastTaskClusterAttempt(tc);
+                            if (lastTaskClusterAttempt != null
+                                    && (lastTaskClusterAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED || lastTaskClusterAttempt
+                                            .getStatus() == TaskClusterAttempt.TaskClusterStatus.RUNNING)) {
+                                boolean abort = false;
+                                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,
+                                                Collections.singletonList(new Exception("Node " + ta.getNodeId()
+                                                        + " failed")));
+                                        ta.setEndTime(System.currentTimeMillis());
+                                        abort = true;
+                                    }
+                                }
+                                if (abort) {
+                                    abortTaskCluster(lastTaskClusterAttempt,
+                                            TaskClusterAttempt.TaskClusterStatus.ABORTED);
                                 }
                             }
-                            if (abort) {
-                                abortTaskCluster(lastTaskClusterAttempt);
-                            }
                         }
+                        abortDoomedTaskClusters();
                     }
-                    abortDoomedTaskClusters();
                 }
             }
             startRunnableActivityClusters();
@@ -668,4 +683,37 @@
             abortJob(Collections.singletonList(e));
         }
     }
+
+    private void finishJob(final JobRun run) {
+        JobId jobId = run.getJobId();
+        CCApplicationContext appCtx = ccs.getApplicationContext();
+        if (appCtx != null) {
+            try {
+                appCtx.notifyJobFinish(jobId);
+            } catch (HyracksException e) {
+                e.printStackTrace();
+            }
+        }
+        run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
+        ccs.getActiveRunMap().remove(jobId);
+        ccs.getRunMapArchive().put(jobId, run);
+        ccs.getRunHistory().put(jobId, run.getExceptions());
+        try {
+            ccs.getJobLogFile().log(createJobLogObject(run));
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private JSONObject createJobLogObject(final JobRun run) {
+        JSONObject jobLogObject = new JSONObject();
+        try {
+            ActivityClusterGraph acg = run.getActivityClusterGraph();
+            jobLogObject.put("activity-cluster-graph", acg.toJSON());
+            jobLogObject.put("job-run", run.toJSON());
+        } catch (JSONException e) {
+            throw new RuntimeException(e);
+        }
+        return jobLogObject;
+    }
 }
\ 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/AbstractHeartbeatWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/AbstractHeartbeatWork.java
new file mode 100644
index 0000000..7605295
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/AbstractHeartbeatWork.java
@@ -0,0 +1,49 @@
+/*
+ * 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.control.cc.work;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public abstract class AbstractHeartbeatWork extends SynchronizableWork {
+
+    private final ClusterControllerService ccs;
+    private final String nodeId;
+    private final HeartbeatData hbData;
+
+    public AbstractHeartbeatWork(ClusterControllerService ccs, String nodeId, HeartbeatData hbData) {
+        this.ccs = ccs;
+        this.nodeId = nodeId;
+        this.hbData = hbData;
+    }
+
+    @Override
+    public void doRun() {
+        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
+        NodeControllerState state = nodeMap.get(nodeId);
+        if (state != null) {
+            state.notifyHeartbeat(hbData);
+        }
+        runWork();
+    }
+
+    public abstract void runWork();
+
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/AbstractTaskLifecycleWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
index 58aaa57..bcb278b 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/AbstractTaskLifecycleWork.java
@@ -29,15 +29,15 @@
 import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
 import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
 import edu.uci.ics.hyracks.control.cc.job.TaskClusterAttempt;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
-public abstract class AbstractTaskLifecycleWork extends AbstractWork {
+public abstract class AbstractTaskLifecycleWork extends AbstractHeartbeatWork {
     protected final ClusterControllerService ccs;
     protected final JobId jobId;
     protected final TaskAttemptId taId;
     protected final String nodeId;
 
     public AbstractTaskLifecycleWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
+        super(ccs, nodeId, null);
         this.ccs = ccs;
         this.jobId = jobId;
         this.taId = taId;
@@ -45,7 +45,7 @@
     }
 
     @Override
-    public final void run() {
+    public final void runWork() {
         JobRun run = ccs.getActiveRunMap().get(jobId);
         if (run != null) {
             TaskId tid = taId.getTaskId();
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
index bc58d1e..c4d202f 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ApplicationMessageWork.java
@@ -22,12 +22,11 @@
 import edu.uci.ics.hyracks.api.messages.IMessage;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
 /**
  * @author rico
  */
-public class ApplicationMessageWork extends AbstractWork {
+public class ApplicationMessageWork extends AbstractHeartbeatWork {
 
     private static final Logger LOGGER = Logger.getLogger(ApplicationMessageWork.class.getName());
     private byte[] message;
@@ -36,6 +35,7 @@
     private ClusterControllerService ccs;
 
     public ApplicationMessageWork(ClusterControllerService ccs, byte[] message, DeploymentId deploymentId, String nodeId) {
+        super(ccs, nodeId, null);
         this.ccs = ccs;
         this.deploymentId = deploymentId;
         this.nodeId = nodeId;
@@ -43,7 +43,7 @@
     }
 
     @Override
-    public void run() {
+    public void runWork() {
         final ICCApplicationContext ctx = ccs.getApplicationContext();
         try {
             final IMessage data = (IMessage) DeploymentUtils.deserialize(message, deploymentId, ctx);
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 6e8ddf0..46a7c16 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.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.logging.Logger;
@@ -48,11 +49,16 @@
 
     @Override
     public void run() {
+        LOGGER.info("Cleanup for JobRun with id: " + jobId);
         final JobRun run = ccs.getActiveRunMap().get(jobId);
         if (run == null) {
             LOGGER.warning("Unable to find JobRun with id: " + jobId);
             return;
         }
+        if (run.getPendingStatus() != null && run.getCleanupPendingNodeIds().isEmpty()) {
+            finishJob(run);
+            return;
+        }
         if (run.getPendingStatus() != null) {
             LOGGER.warning("Ignoring duplicate cleanup for JobRun with id: " + jobId);
             return;
@@ -63,33 +69,47 @@
             run.setPendingStatus(status, exceptions);
         }
         if (targetNodes != null && !targetNodes.isEmpty()) {
+            Set<String> toDelete = new HashSet<String>();
             for (String n : targetNodes) {
                 NodeControllerState ncs = ccs.getNodeMap().get(n);
                 try {
-                    ncs.getNodeController().cleanUpJoblet(jobId, status);
+                    if (ncs == null) {
+                        toDelete.add(n);
+                    } else {
+                        ncs.getNodeController().cleanUpJoblet(jobId, status);
+                    }
                 } catch (Exception e) {
                     e.printStackTrace();
                 }
             }
+            targetNodes.removeAll(toDelete);
+            run.getCleanupPendingNodeIds().removeAll(toDelete);
+            if (run.getCleanupPendingNodeIds().isEmpty()) {
+                finishJob(run);
+            }
         } else {
-            CCApplicationContext appCtx = ccs.getApplicationContext();
-            if (appCtx != null) {
-                try {
-                    appCtx.notifyJobFinish(jobId);
-                } catch (HyracksException e) {
-                    e.printStackTrace();
-                }
-            }
-            run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
-            ccs.getActiveRunMap().remove(jobId);
-            ccs.getRunMapArchive().put(jobId, run);
-            ccs.getRunHistory().put(jobId, run.getExceptions());
+            finishJob(run);
+        }
+    }
+
+    private void finishJob(final JobRun run) {
+        CCApplicationContext appCtx = ccs.getApplicationContext();
+        if (appCtx != null) {
             try {
-                ccs.getJobLogFile().log(createJobLogObject(run));
-            } catch (Exception e) {
-                throw new RuntimeException(e);
+                appCtx.notifyJobFinish(jobId);
+            } catch (HyracksException e) {
+                e.printStackTrace();
             }
         }
+        run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
+        ccs.getActiveRunMap().remove(jobId);
+        ccs.getRunMapArchive().put(jobId, run);
+        ccs.getRunHistory().put(jobId, run.getExceptions());
+        try {
+            ccs.getJobLogFile().log(createJobLogObject(run));
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
     }
 
     private JSONObject createJobLogObject(final JobRun run) {
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 5eb851a..2d6bdea 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
@@ -28,9 +28,8 @@
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
 import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
 import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
-public class JobletCleanupNotificationWork extends AbstractWork {
+public class JobletCleanupNotificationWork extends AbstractHeartbeatWork {
     private static final Logger LOGGER = Logger.getLogger(JobletCleanupNotificationWork.class.getName());
 
     private ClusterControllerService ccs;
@@ -38,13 +37,14 @@
     private String nodeId;
 
     public JobletCleanupNotificationWork(ClusterControllerService ccs, JobId jobId, String nodeId) {
+        super(ccs, nodeId, null);
         this.ccs = ccs;
         this.jobId = jobId;
         this.nodeId = nodeId;
     }
 
     @Override
-    public void run() {
+    public void runWork() {
         final JobRun run = ccs.getActiveRunMap().get(jobId);
         Set<String> cleanupPendingNodes = run.getCleanupPendingNodeIds();
         if (!cleanupPendingNodes.remove(nodeId)) {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NodeHeartbeatWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NodeHeartbeatWork.java
index 970a45d..8ef8f66 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NodeHeartbeatWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NodeHeartbeatWork.java
@@ -14,32 +14,20 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
-import java.util.Map;
 import java.util.logging.Level;
 
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.NodeControllerState;
 import edu.uci.ics.hyracks.control.common.heartbeat.HeartbeatData;
-import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
 
-public class NodeHeartbeatWork extends SynchronizableWork {
-    private final ClusterControllerService ccs;
-    private final String nodeId;
-    private final HeartbeatData hbData;
+public class NodeHeartbeatWork extends AbstractHeartbeatWork {
 
     public NodeHeartbeatWork(ClusterControllerService ccs, String nodeId, HeartbeatData hbData) {
-        this.ccs = ccs;
-        this.nodeId = nodeId;
-        this.hbData = hbData;
+        super(ccs, nodeId, hbData);
     }
 
     @Override
-    protected void doRun() throws Exception {
-        Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
-        NodeControllerState state = nodeMap.get(nodeId);
-        if (state != null) {
-            state.notifyHeartbeat(hbData);
-        }
+    public void runWork() {
+
     }
 
     @Override
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NotifyDeployBinaryWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NotifyDeployBinaryWork.java
index c4c8873..c35f385 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NotifyDeployBinaryWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/NotifyDeployBinaryWork.java
@@ -19,14 +19,13 @@
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.common.deployment.DeploymentRun;
 import edu.uci.ics.hyracks.control.common.deployment.DeploymentStatus;
-import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 
 /***
  * This is the work happens on the CC when CC gets a deployment or undeployment notification status message from one NC.
  * 
  * @author yingyib
  */
-public class NotifyDeployBinaryWork extends AbstractWork {
+public class NotifyDeployBinaryWork extends AbstractHeartbeatWork {
 
     private final ClusterControllerService ccs;
     private final String nodeId;
@@ -35,6 +34,7 @@
 
     public NotifyDeployBinaryWork(ClusterControllerService ccs, DeploymentId deploymentId, String nodeId,
             DeploymentStatus deploymentStatus) {
+        super(ccs, nodeId, null);
         this.ccs = ccs;
         this.nodeId = nodeId;
         this.deploymentId = deploymentId;
@@ -43,7 +43,7 @@
     }
 
     @Override
-    public void run() {
+    public void runWork() {
         /** triggered remotely by a NC to notify that the NC is deployed */
         DeploymentRun dRun = ccs.getDeploymentRun(deploymentId);
         dRun.notifyDeploymentStatus(nodeId, deploymentStatus);
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/dataset/ResultStateSweeper.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/dataset/ResultStateSweeper.java
index 69b560c..5a6d849 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/dataset/ResultStateSweeper.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/dataset/ResultStateSweeper.java
@@ -55,6 +55,7 @@
             } catch (InterruptedException e) {
                 LOGGER.severe("Result cleaner thread interrupted, but we continue running it.");
                 // There isn't much we can do really here
+                break; // the interrupt was explicit from another thread. This thread should shut down...
             }
         }
 
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ExceptionFilterUtils.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/utils/ExceptionUtils.java
similarity index 69%
rename from hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ExceptionFilterUtils.java
rename to hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/utils/ExceptionUtils.java
index 44f3bec..cbdc6e5 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ExceptionFilterUtils.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/utils/ExceptionUtils.java
@@ -12,15 +12,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.control.cc.job;
+package edu.uci.ics.hyracks.control.common.utils;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
 /**
  * @author yingyib
  */
-public class ExceptionFilterUtils {
+public class ExceptionUtils {
 
     public static List<Exception> getActualExceptions(List<Exception> allExceptions) {
         List<Exception> exceptions = new ArrayList<Exception>();
@@ -32,6 +35,17 @@
         return exceptions;
     }
 
+    public static void setNodeIds(Collection<Exception> exceptions, String nodeId) {
+        List<Exception> newExceptions = new ArrayList<Exception>();
+        for (Exception e : exceptions) {
+            HyracksDataException newException = new HyracksDataException(e);
+            newException.setNodeId(nodeId);
+            newExceptions.add(newException);
+        }
+        exceptions.clear();
+        exceptions.addAll(newExceptions);
+    }
+
     private static boolean possibleRootCause(Throwable exception) {
         Throwable cause = exception;
         while ((cause = cause.getCause()) != null) {
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
index f12c981..58e12cf 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/work/WorkQueue.java
@@ -87,6 +87,7 @@
     private class WorkerThread extends Thread {
         WorkerThread() {
             setDaemon(true);
+            setPriority(MAX_PRIORITY);
         }
 
         @Override
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 53e5a01..c72ced1 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
@@ -49,6 +49,7 @@
 import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.PartitionProfile;
 import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+import edu.uci.ics.hyracks.control.common.utils.ExceptionUtils;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
@@ -243,6 +244,7 @@
                                 addPendingThread(thread);
                                 String oldName = thread.getName();
                                 thread.setName(displayName + ":" + taskAttemptId + ":" + cIdx);
+                                thread.setPriority(Thread.MIN_PRIORITY);
                                 try {
                                     pushFrames(collector, writer);
                                 } catch (HyracksDataException e) {
@@ -276,7 +278,11 @@
             removePendingThread(ct);
         }
         if (!exceptions.isEmpty()) {
+            for (Exception e : exceptions) {
+                e.printStackTrace();
+            }
             NodeControllerService ncs = joblet.getNodeController();
+            ExceptionUtils.setNodeIds(exceptions, ncs.getId());
             ncs.getWorkQueue().schedule(new NotifyTaskFailureWork(ncs, this, exceptions));
         }
     }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
index ce4c74c..ac44c11 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 
 /**
  * FrameTupleCursor is used to navigate over tuples in a Frame. A frame is
@@ -57,27 +58,28 @@
 
     @Override
     public int getTupleCount() {
-        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
+        return IntSerDeUtils.getInt(buffer.array(), FrameHelper.getTupleCountOffset(frameSize));
     }
 
     @Override
     public int getTupleStartOffset(int tupleIndex) {
-        return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
+        return tupleIndex == 0 ? 0 : IntSerDeUtils.getInt(buffer.array(), FrameHelper.getTupleCountOffset(frameSize)
+                - 4 * tupleIndex);
     }
 
     @Override
     public int getTupleEndOffset(int tupleIndex) {
-        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
+        return IntSerDeUtils.getInt(buffer.array(), FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
     }
 
     @Override
     public int getFieldStartOffset(int tupleIndex, int fIdx) {
-        return fIdx == 0 ? 0 : buffer.getInt(getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+        return fIdx == 0 ? 0 : IntSerDeUtils.getInt(buffer.array(), getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
     }
 
     @Override
     public int getFieldEndOffset(int tupleIndex, int fIdx) {
-        return buffer.getInt(getTupleStartOffset(tupleIndex) + fIdx * 4);
+        return IntSerDeUtils.getInt(buffer.array(), getTupleStartOffset(tupleIndex) + fIdx * 4);
     }
 
     @Override
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java
new file mode 100644
index 0000000..9faef09
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java
@@ -0,0 +1,25 @@
+/*
+ * 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.common.util;
+
+public class IntSerDeUtils {
+
+    public static int getInt(byte[] bytes, int offset) {
+        return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
+                + ((bytes[offset + 3] & 0xff) << 0);
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
index 30b2482..df4d296 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
@@ -38,4 +38,9 @@
         sourceBitmap.clear();
         sourceBitmap.set(0, nProducerPartitions);
     }
+    
+    @Override
+    public boolean allProducersToAllConsumers(){
+        return true;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
index 466fead..20a0ed1 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
@@ -82,4 +82,9 @@
         sourceBitmap.clear();
         sourceBitmap.set(consumerIndex);
     }
+
+    @Override
+    public boolean allProducersToAllConsumers() {
+        return false;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
index f71ee1d..a6bb4e2 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 
 public class FrameSorter {
     private final IHyracksTaskContext ctx;
@@ -167,7 +168,7 @@
         }
     }
 
-    /** Merge two subarrays into one*/
+    /** Merge two subarrays into one */
     private void merge(int start1, int start2, int len1, int len2) {
         int targetPos = start1;
         int pos1 = start1;
@@ -224,12 +225,12 @@
         fta2.reset(buf2);
         for (int f = 0; f < comparators.length; ++f) {
             int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : buf1.getInt(j1 + (fIdx - 1) * 4);
-            int f1End = buf1.getInt(j1 + fIdx * 4);
+            int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf1.array(), j1 + (fIdx - 1) * 4);
+            int f1End = IntSerDeUtils.getInt(buf1.array(), j1 + fIdx * 4);
             int s1 = j1 + fta1.getFieldSlotsLength() + f1Start;
             int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : buf2.getInt(j2 + (fIdx - 1) * 4);
-            int f2End = buf2.getInt(j2 + fIdx * 4);
+            int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf2.array(), j2 + (fIdx - 1) * 4);
+            int f2End = IntSerDeUtils.getInt(buf2.array(), j2 + fIdx * 4);
             int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
             int l2 = f2End - f2Start;
             int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java
index a45992c..814be7a 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java
@@ -98,8 +98,9 @@
             public void initialize() throws HyracksDataException {
                 ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
                 try {
-                    Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
+                    Thread.currentThread().setContextClassLoader(ctx.getJobletContext().getClassLoader());
                     JobConf conf = confFactory.getConf();
+                    conf.setClassLoader(ctx.getJobletContext().getClassLoader());
                     IKeyValueParser parser = tupleParserFactory.createKeyValueParser(ctx);
                     writer.open();
                     parser.open(writer);
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
index 43ca4ac..bc47360 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
@@ -112,7 +112,7 @@
             public void initialize() throws HyracksDataException {
                 ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
                 try {
-                    Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+                    Thread.currentThread().setContextClassLoader(ctx.getJobletContext().getClassLoader());
                     Job job = confFactory.getConf();
                     job.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
                     IKeyValueParser parser = tupleParserFactory.createKeyValueParser(ctx);
@@ -141,6 +141,7 @@
                              * read the split
                              */
                             TaskAttemptContext context = ctxFactory.createContext(job.getConfiguration(), i);
+                            context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
                             RecordReader reader = inputFormat.createRecordReader(inputSplits.get(i), context);
                             reader.initialize(inputSplits.get(i), context);
                             while (reader.nextKeyValue() == true) {
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
index 75553e1..85f80ac 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
 import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
 
 /**
@@ -54,7 +55,20 @@
     public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
         scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ncNameToNcInfos);
     }
-    
+
+    /**
+     * The constructor of the scheduler.
+     * 
+     * @param ncNameToNcInfos
+     *            the mapping from nc names to nc infos
+     * @param topology
+     *            the hyracks cluster toplogy
+     * @throws HyracksException
+     */
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, ClusterTopology topology) throws HyracksException {
+        scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ncNameToNcInfos, topology);
+    }
+
     /**
      * The constructor of the scheduler.
      * 
@@ -62,7 +76,8 @@
      *            the mapping from nc names to nc infos
      * @throws HyracksException
      */
-    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder builder) throws HyracksException {
+    public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder builder)
+            throws HyracksException {
         scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ncNameToNcInfos, builder);
     }
 
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
index fbece8b..fe25db8 100644
--- a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
@@ -47,7 +47,8 @@
     private int numHashes;
     private long numElements;
     private long numBits;
-    private int numBitsPerPage;
+    private final int numBitsPerPage;
+    private final static byte[] ZERO_BUFFER = new byte[131072]; // 128kb
 
     private final ArrayList<ICachedPage> bloomFilterPages = new ArrayList<ICachedPage>();
     private final static long SEED = 0L;
@@ -58,7 +59,7 @@
         this.fileMapProvider = fileMapProvider;
         this.file = file;
         this.keyFields = keyFields;
-        numBitsPerPage = bufferCache.getPageSize() * Byte.SIZE;
+        this.numBitsPerPage = bufferCache.getPageSize() * Byte.SIZE;
     }
 
     public int getFileId() {
@@ -198,7 +199,6 @@
 
     public class BloomFilterBuilder implements IIndexBulkLoader {
         private final long[] hashes = new long[2];
-
         private final long numElements;
         private final int numHashes;
         private final long numBits;
@@ -223,11 +223,25 @@
             while (currentPageId <= numPages) {
                 ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
                 page.acquireWriteLatch();
+                initPage(page.getBuffer().array());
                 bloomFilterPages.add(page);
                 ++currentPageId;
             }
         }
 
+        private void initPage(byte[] array) {
+            int numRounds = array.length / ZERO_BUFFER.length;
+            int leftOver = array.length % ZERO_BUFFER.length;
+            int destPos = 0;
+            for (int i = 0; i < numRounds; i++) {
+                System.arraycopy(ZERO_BUFFER, 0, array, destPos, ZERO_BUFFER.length);
+                destPos = (i + 1) * ZERO_BUFFER.length;
+            }
+            if (leftOver > 0) {
+                System.arraycopy(ZERO_BUFFER, 0, array, destPos, leftOver);
+            }
+        }
+
         private void persistBloomFilterMetaData() throws HyracksDataException {
             ICachedPage metaPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, METADATA_PAGE_ID), false);
             metaPage.acquireWriteLatch();
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index a85a174..c14f23a 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -317,7 +317,11 @@
 
                 if (!releasedLatches) {
                     for (int i = 0; i < nodeFrontiers.size(); i++) {
-                        nodeFrontiers.get(i).page.releaseWriteLatch();
+                        try {
+                            nodeFrontiers.get(i).page.releaseWriteLatch();
+                        } catch (Exception e) {
+                            //ignore illegal monitor state exception
+                        }
                         bufferCache.unpin(nodeFrontiers.get(i).page);
                     }
                 }
diff --git a/pregelix/pom.xml b/pregelix/pom.xml
index de5ef3b..d748d1f 100644
--- a/pregelix/pom.xml
+++ b/pregelix/pom.xml
@@ -12,8 +12,7 @@
  ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  ! See the License for the specific language governing permissions and
  ! limitations under the License.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ !--><project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <groupId>edu.uci.ics.hyracks</groupId>
   <artifactId>pregelix</artifactId>
@@ -22,7 +21,7 @@
   <name>pregelix</name>
 
   <properties>
-    <jvm.extraargs />
+    <jvm.extraargs/>
   </properties>
 
   <profiles>
@@ -107,6 +106,7 @@
     <module>pregelix-runtime</module>
     <module>pregelix-core</module>
     <module>pregelix-example</module>
+    <module>pregelix-benchmark</module>
     <module>pregelix-dist</module>
   </modules>
 </project>
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
index 1e0d87a..6549c52 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
@@ -76,6 +76,10 @@
     public static final String UPDATE_INTENSIVE = "pregelix.updateIntensive";
     /** the check point hook */
     public static final String CKP_CLASS = "pregelix.checkpointHook";
+    /** the check point hook */
+    public static final String RECOVERY_COUNT = "pregelix.recoveryCount";
+    /** the checkpoint interval */
+    public static final String CKP_INTERVAL = "pregelix.ckpinterval";
 
     /**
      * Construct a Pregelix job from an existing configuration
@@ -223,6 +227,24 @@
         getConfiguration().setClass(CKP_CLASS, ckpClass, ICheckpointHook.class);
     }
 
+    /**
+     * Users can provide an ICheckpointHook implementation to specify when to do checkpoint
+     * 
+     * @param ckpClass
+     */
+    final public void setRecoveryCount(int recoveryCount) {
+        getConfiguration().setInt(RECOVERY_COUNT, recoveryCount);
+    }
+
+    /**
+     * Users can set the interval of checkpointing
+     * 
+     * @param ckpInterval
+     */
+    final public void setCheckpointingInterval(int ckpInterval) {
+        getConfiguration().setInt(CKP_INTERVAL, ckpInterval);
+    }
+
     @Override
     public String toString() {
         return getJobName();
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
index d68ad2c..4ee1deb 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
@@ -549,4 +549,57 @@
     public static String getGlobalAggregateSpillingDirName(Configuration conf, long superStep) {
         return "/tmp/pregelix/agg/" + conf.get(PregelixJob.JOB_ID) + "/" + superStep;
     }
+
+    /**
+     * Get the path for vertex checkpointing
+     * 
+     * @param conf
+     * @param lastSuperStep
+     * @return the path for vertex checkpointing
+     */
+    public static String getVertexCheckpointPath(Configuration conf, long lastSuperStep) {
+        return "/tmp/ckpoint/" + BspUtils.getJobId(conf) + "/vertex/" + lastSuperStep;
+    }
+
+    /**
+     * Get the path for message checkpointing
+     * 
+     * @param conf
+     * @param lastSuperStep
+     * @return the path for message checkpointing
+     */
+    public static String getMessageCheckpointPath(Configuration conf, long lastSuperStep) {
+        String path = "/tmp/ckpoint/" + BspUtils.getJobId(conf) + "/message/" + lastSuperStep;
+        return path;
+    }
+
+    /**
+     * Get the path for message checkpointing
+     * 
+     * @param conf
+     * @param lastSuperStep
+     * @return the path for message checkpointing
+     */
+    public static String getSecondaryIndexCheckpointPath(Configuration conf, long lastSuperStep) {
+        return "/tmp/ckpoint/" + BspUtils.getJobId(conf) + "/secondaryindex/" + lastSuperStep;
+    }
+
+    /***
+     * Get the recovery count
+     * 
+     * @return recovery count
+     */
+    public static int getRecoveryCount(Configuration conf) {
+        return conf.getInt(PregelixJob.RECOVERY_COUNT, 0);
+    }
+
+    /***
+     * Get the user-set checkpoint interval
+     * 
+     * @param conf
+     * @return the checkpoint interval
+     */
+    public static int getCheckpointingInterval(Configuration conf) {
+        return conf.getInt(PregelixJob.CKP_INTERVAL, -1);
+    }
 }
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ConservativeCheckpointHook.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ConservativeCheckpointHook.java
index 6a4a660..4f5fef0 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ConservativeCheckpointHook.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ConservativeCheckpointHook.java
@@ -25,7 +25,7 @@
 
     @Override
     public boolean checkpoint(int superstep) {
-        if (superstep % 5 == 0) {
+        if (superstep % 2 == 0) {
             return true;
         } else {
             return false;
diff --git a/pregelix/pregelix-benchmark/pom.xml b/pregelix/pregelix-benchmark/pom.xml
new file mode 100644
index 0000000..4d7d456
--- /dev/null
+++ b/pregelix/pregelix-benchmark/pom.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0"?>
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<artifactId>pregelix</artifactId>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<version>0.2.10-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>pregelix-benchmark</artifactId>
+	<name>pregelix-benchmark</name>
+	<url>http://maven.apache.org</url>
+	<dependencies>
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>3.8.1</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.giraph</groupId>
+			<artifactId>giraph-core</artifactId>
+			<version>1.0.0</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-hdfs-core</artifactId>
+			<version>0.2.10-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+	</dependencies>
+</project>
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
new file mode 100644
index 0000000..04c29de
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.pregelix.benchmark;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.VLongWritable;
+
+/**
+ * Demonstrates the basic Pregel PageRank implementation.
+ */
+public class PageRankVertex extends Vertex<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
+
+    public static final String ITERATIONS = "HyracksPageRankVertex.iteration";
+    private final DoubleWritable vertexValue = new DoubleWritable();
+    private final DoubleWritable msg = new DoubleWritable();
+    private int maxIteration = -1;
+
+    @Override
+    public void compute(Iterable<DoubleWritable> msgIterator) {
+        if (maxIteration < 0) {
+            maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 10);
+        }
+        if (getSuperstep() == 1) {
+            vertexValue.set(1.0 / getTotalNumVertices());
+        }
+        if (getSuperstep() >= 2 && getSuperstep() <= maxIteration) {
+            double sum = 0;
+            for (DoubleWritable msg : msgIterator) {
+                sum += msg.get();
+            }
+            vertexValue.set((0.15 / getTotalNumVertices()) + 0.85 * sum);
+        }
+
+        if (getSuperstep() >= 1 && getSuperstep() < maxIteration) {
+            long edges = getNumEdges();
+            msg.set(vertexValue.get() / edges);
+            sendMessageToAllEdges(msg);
+        } else {
+            voteToHalt();
+        }
+    }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
new file mode 100644
index 0000000..3d85f66
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.benchmark;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.MapMutableEdge;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.VLongWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+
+    @Override
+    public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
+        return new TextVertexReaderFromEachLine() {
+            String[] items;
+
+            @Override
+            protected VLongWritable getId(Text line) throws IOException {
+                items = line.toString().split(" ");
+                return new VLongWritable(Long.parseLong(items[0]));
+            }
+
+            @Override
+            protected DoubleWritable getValue(Text line) throws IOException {
+                return null;
+            }
+
+            @Override
+            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
+                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
+                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+                for (int i = 1; i < items.length; i++) {
+                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                }
+                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                    edge.setEntry(entry);
+                    edge.setValue(null);
+                    edges.add(edge);
+                }
+                return edges;
+            }
+
+        };
+    }
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
index 7bd2cf8..d6a6f3d 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
@@ -23,7 +23,9 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -50,6 +52,7 @@
 import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSingleSort;
 import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSort;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
+import edu.uci.ics.pregelix.core.util.ExceptionUtilities;
 import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
 
 @SuppressWarnings("rawtypes")
@@ -89,6 +92,7 @@
             this.profiling = profiling;
             PregelixJob currentJob = jobs.get(0);
             PregelixJob lastJob = currentJob;
+            addHadoopConfiguration(currentJob, ipAddress, port, true);
             JobGen jobGen = null;
 
             /** prepare job -- deploy jars */
@@ -99,27 +103,29 @@
             IntWritable lastSnapshotSuperstep = new IntWritable(0);
             boolean failed = false;
             int retryCount = 0;
-            int maxRetryCount = 1;
+            int maxRetryCount = 3;
+            jobGen = selectJobGen(planChoice, currentJob);
 
             do {
                 try {
                     for (int i = lastSnapshotJobIndex.get(); i < jobs.size(); i++) {
                         lastJob = currentJob;
                         currentJob = jobs.get(i);
+                        currentJob.setRecoveryCount(retryCount);
 
                         /** add hadoop configurations */
-                        addHadoopConfiguration(currentJob, ipAddress, port);
+                        addHadoopConfiguration(currentJob, ipAddress, port, failed);
                         ICheckpointHook ckpHook = BspUtils.createCheckpointHook(currentJob.getConfiguration());
 
                         /** load the data */
-                        if (i == 0 || compatible(lastJob, currentJob)) {
+                        if ((i == 0 || compatible(lastJob, currentJob)) && !failed) {
                             if (i != 0) {
                                 finishJobs(jobGen, deploymentId);
                                 /** invalidate/clear checkpoint */
                                 lastSnapshotJobIndex.set(0);
                                 lastSnapshotSuperstep.set(0);
                             }
-                            jobGen = selectJobGen(planChoice, currentJob);
+                            jobGen.reset(currentJob);
                             loadData(currentJob, jobGen, deploymentId);
                         } else {
                             jobGen.reset(currentJob);
@@ -137,12 +143,16 @@
                     /** clear checkpoints if any */
                     jobGen.clearCheckpoints();
                     hcc.unDeployBinary(deploymentId);
-                } catch (IOException ioe) {
-                    /** disk failures */
-                    //restart from snapshot
-                    failed = true;
-                    retryCount++;
-                    throw new HyracksException(ioe);
+                } catch (Exception e1) {
+                    Set<String> blackListNodes = new HashSet<String>();
+                    /** disk failures or node failures */
+                    if (ExceptionUtilities.recoverable(e1, blackListNodes)) {
+                        ClusterConfig.addToBlackListNodes(blackListNodes);
+                        failed = true;
+                        retryCount++;
+                    } else {
+                        throw e1;
+                    }
                 }
             } while (failed && retryCount < maxRetryCount);
             LOG.info("job finished");
@@ -222,9 +232,9 @@
     }
 
     private DeploymentId prepareJobs(String ipAddress, int port) throws Exception {
-        if (hcc == null)
+        if (hcc == null) {
             hcc = new HyracksConnection(ipAddress, port);
-
+        }
         URLClassLoader classLoader = (URLClassLoader) exampleClass.getClassLoader();
         List<File> jars = new ArrayList<File>();
         URL[] urls = classLoader.getURLs();
@@ -235,7 +245,8 @@
         return deploymentId;
     }
 
-    private void addHadoopConfiguration(PregelixJob job, String ipAddress, int port) throws HyracksException {
+    private void addHadoopConfiguration(PregelixJob job, String ipAddress, int port, boolean loadClusterConfig)
+            throws HyracksException {
         URL hadoopCore = job.getClass().getClassLoader().getResource("core-site.xml");
         if (hadoopCore != null) {
             job.getConfiguration().addResource(hadoopCore);
@@ -248,7 +259,9 @@
         if (hadoopHdfs != null) {
             job.getConfiguration().addResource(hadoopHdfs);
         }
-        ClusterConfig.loadClusterConfig(ipAddress, port);
+        if (loadClusterConfig) {
+            ClusterConfig.loadClusterConfig(ipAddress, port);
+        }
     }
 
     private void runLoopBody(DeploymentId deploymentId, PregelixJob job, JobGen jobGen, int currentJobIndex,
@@ -256,10 +269,16 @@
             throws Exception {
         if (doRecovery) {
             /** reload the checkpoint */
-            runLoadCheckpoint(deploymentId, jobGen, snapshotSuperstep.get());
-
+            if (snapshotSuperstep.get() > 0) {
+                runClearState(deploymentId, jobGen);
+                runLoadCheckpoint(deploymentId, jobGen, snapshotSuperstep.get());
+            } else {
+                runClearState(deploymentId, jobGen);
+                loadData(job, jobGen, deploymentId);
+            }
         }
         int i = doRecovery ? snapshotSuperstep.get() + 1 : 1;
+        int ckpInterval = BspUtils.getCheckpointingInterval(job.getConfiguration());
         boolean terminate = false;
         long start, end, time;
         do {
@@ -270,10 +289,10 @@
             LOG.info(job + ": iteration " + i + " finished " + time + "ms");
             terminate = IterationUtils.readTerminationState(job.getConfiguration(), jobGen.getJobId())
                     || IterationUtils.readForceTerminationState(job.getConfiguration(), jobGen.getJobId());
-            if (ckpHook.checkpoint(i)) {
+            if (ckpHook.checkpoint(i) || (ckpInterval > 0 && i % ckpInterval == 0)) {
                 runCheckpoint(deploymentId, jobGen, i);
-                snapshotSuperstep.set(i);
                 snapshotJobIndex.set(currentJobIndex);
+                snapshotSuperstep.set(i);
             }
             i++;
         } while (!terminate);
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 36723a6..4863378 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
@@ -36,9 +36,12 @@
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -61,10 +64,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
-import edu.uci.ics.hyracks.hdfs2.dataflow.ConfFactory;
 import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSWriteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
@@ -102,10 +102,10 @@
 import edu.uci.ics.pregelix.dataflow.EmptyTupleSourceOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.HDFSFileWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.KeyValueParserFactory;
-import edu.uci.ics.pregelix.dataflow.KeyValueWriterFactory;
 import edu.uci.ics.pregelix.dataflow.MaterializingReadOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.VertexFileScanOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.VertexFileWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.VertexWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
 import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
@@ -137,8 +137,6 @@
     protected static final String SECONDARY_INDEX_ODD = "secondary1";
     protected static final String SECONDARY_INDEX_EVEN = "secondary2";
 
-    private String vertexCheckpointPath;
-
     public JobGen(PregelixJob job) {
         init(job);
     }
@@ -148,10 +146,7 @@
         pregelixJob = job;
         initJobConfiguration();
         job.setJobId(jobId);
-
-        vertexCheckpointPath = "/tmp/ckpoint/" + jobId + "/vertex";
-        // set the frame size to be the one user specified if the user did
-        // specify.
+        // set the frame size to be the one user specified if the user did specify.
         int specifiedFrameSize = BspUtils.getFrameSize(job.getConfiguration());
         if (specifiedFrameSize > 0) {
             frameSize = specifiedFrameSize;
@@ -376,15 +371,21 @@
     @Override
     public JobSpecification[] generateCheckpointing(int lastSuccessfulIteration) throws HyracksException {
         try {
-
             PregelixJob tmpJob = this.createCloneJob("Vertex checkpointing for job " + jobId, pregelixJob);
             tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
-            FileOutputFormat.setOutputPath(tmpJob, new Path(vertexCheckpointPath + "/" + lastSuccessfulIteration));
+            FileOutputFormat.setOutputPath(tmpJob,
+                    new Path(BspUtils.getVertexCheckpointPath(conf, lastSuccessfulIteration)));
             tmpJob.setOutputKeyClass(NullWritable.class);
             tmpJob.setOutputValueClass(BspUtils.getVertexClass(tmpJob.getConfiguration()));
+            FileSystem dfs = FileSystem.get(tmpJob.getConfiguration());
+
+            dfs.delete(new Path(BspUtils.getVertexCheckpointPath(conf, lastSuccessfulIteration)), true);
             JobSpecification vertexCkpSpec = scanIndexWriteToHDFS(tmpJob.getConfiguration());
+
+            dfs.delete(new Path(BspUtils.getMessageCheckpointPath(conf, lastSuccessfulIteration)), true);
             JobSpecification[] stateCkpSpecs = generateStateCheckpointing(lastSuccessfulIteration);
             JobSpecification[] specs = new JobSpecification[1 + stateCkpSpecs.length];
+
             specs[0] = vertexCkpSpec;
             for (int i = 1; i < specs.length; i++) {
                 specs[i] = stateCkpSpecs[i - 1];
@@ -397,7 +398,7 @@
 
     @Override
     public JobSpecification generateLoadingJob() throws HyracksException {
-        JobSpecification spec = loadHDFSData(conf);
+        JobSpecification spec = loadHDFSData(pregelixJob);
         return spec;
     }
 
@@ -412,13 +413,22 @@
         try {
             PregelixJob tmpJob = this.createCloneJob("Vertex checkpoint loading for job " + jobId, pregelixJob);
             tmpJob.setVertexInputFormatClass(InternalVertexInputFormat.class);
-            FileInputFormat.setInputPaths(tmpJob, new Path(vertexCheckpointPath + "/" + lastCheckpointedIteration));
-            JobSpecification vertexLoadSpec = loadHDFSData(tmpJob.getConfiguration());
+            FileInputFormat.setInputPaths(tmpJob,
+                    new Path(BspUtils.getVertexCheckpointPath(conf, lastCheckpointedIteration)));
+            JobSpecification[] cleanVertices = generateCleanup();
+            JobSpecification createIndex = generateCreatingJob();
+            JobSpecification vertexLoadSpec = loadHDFSData(tmpJob);
             JobSpecification[] stateLoadSpecs = generateStateCheckpointLoading(lastCheckpointedIteration, tmpJob);
-            JobSpecification[] specs = new JobSpecification[1 + stateLoadSpecs.length];
-            specs[0] = vertexLoadSpec;
-            for (int i = 1; i < specs.length; i++) {
-                specs[i] = stateLoadSpecs[i - 1];
+            JobSpecification[] specs = new JobSpecification[cleanVertices.length + 2 + stateLoadSpecs.length];
+
+            int i = 0;
+            for (; i < cleanVertices.length; i++) {
+                specs[i] = cleanVertices[i];
+            }
+            specs[i++] = createIndex;
+            specs[i++] = vertexLoadSpec;
+            for (; i < specs.length; i++) {
+                specs[i] = stateLoadSpecs[i - cleanVertices.length - 2];
             }
             return specs;
         } catch (Exception e) {
@@ -479,7 +489,8 @@
     }
 
     @SuppressWarnings({ "rawtypes", "unchecked" })
-    private JobSpecification loadHDFSData(Configuration conf) throws HyracksException, HyracksDataException {
+    private JobSpecification loadHDFSData(PregelixJob job) throws HyracksException, HyracksDataException {
+        Configuration conf = job.getConfiguration();
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
         Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
         JobSpecification spec = new JobSpecification();
@@ -492,7 +503,7 @@
         VertexInputFormat inputFormat = BspUtils.createVertexInputFormat(conf);
         List<InputSplit> splits = new ArrayList<InputSplit>();
         try {
-            splits = inputFormat.getSplits(pregelixJob, fileSplitProvider.getFileSplits().length);
+            splits = inputFormat.getSplits(job, fileSplitProvider.getFileSplits().length);
             LOGGER.info("number of splits: " + splits.size());
             for (InputSplit split : splits)
                 LOGGER.info(split.toString());
@@ -591,7 +602,8 @@
          */
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
                 vertexIdClass.getName(), vertexClass.getName());
-        HDFSFileWriteOperatorDescriptor writer = new HDFSFileWriteOperatorDescriptor(spec, confFactory, inputRdFactory);
+        VertexFileWriteOperatorDescriptor writer = new VertexFileWriteOperatorDescriptor(spec, confFactory,
+                inputRdFactory);
         ClusterConfig.setLocationConstraint(spec, writer);
 
         /**
@@ -637,18 +649,21 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+                false);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
-        String checkpointPath = "/tmp/ckpoint/" + jobId + "/message/" + lastSuccessfulIteration;
+        String checkpointPath = BspUtils.getMessageCheckpointPath(conf, lastSuccessfulIteration);;
         PregelixJob tmpJob = createCloneJob("State checkpointing for job " + jobId, pregelixJob);
-        tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
+        tmpJob.setOutputFormatClass(SequenceFileOutputFormat.class);
         FileOutputFormat.setOutputPath(tmpJob, new Path(checkpointPath));
         tmpJob.setOutputKeyClass(vertexIdClass);
         tmpJob.setOutputValueClass(MsgList.class);
 
-        ITupleWriterFactory writerFactory = new KeyValueWriterFactory(new ConfFactory(tmpJob));
-        HDFSWriteOperatorDescriptor hdfsWriter = new HDFSWriteOperatorDescriptor(spec, tmpJob, writerFactory);
+        IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                vertexIdClass.getName(), MsgList.class.getName());
+        HDFSFileWriteOperatorDescriptor hdfsWriter = new HDFSFileWriteOperatorDescriptor(spec, tmpJob, inputRdFactory);
+        ClusterConfig.setLocationConstraint(spec, hdfsWriter);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, materializeRead, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, hdfsWriter, 0);
@@ -660,25 +675,26 @@
     @SuppressWarnings({ "unchecked", "rawtypes" })
     protected JobSpecification[] generateStateCheckpointLoading(int lastCheckpointedIteration, PregelixJob job)
             throws HyracksException {
-        String checkpointPath = "/tmp/ckpoint/" + jobId + "/message/" + lastCheckpointedIteration;
+        String checkpointPath = BspUtils.getMessageCheckpointPath(job.getConfiguration(), lastCheckpointedIteration);
         PregelixJob tmpJob = createCloneJob("State checkpoint loading for job " + jobId, job);
-        tmpJob.setVertexInputFormatClass(InternalVertexInputFormat.class);
+        tmpJob.setInputFormatClass(SequenceFileInputFormat.class);
         try {
             FileInputFormat.setInputPaths(tmpJob, checkpointPath);
         } catch (IOException e) {
             throw new HyracksException(e);
         }
-        Configuration conf = job.getConfiguration();
+        Configuration conf = tmpJob.getConfiguration();
         Class vertexIdClass = BspUtils.getVertexIndexClass(conf);
         JobSpecification spec = new JobSpecification();
 
         /***
          * HDFS read operator
          */
-        VertexInputFormat inputFormat = BspUtils.createVertexInputFormat(conf);
         List<InputSplit> splits = new ArrayList<InputSplit>();
         try {
-            splits = inputFormat.getSplits(tmpJob, ClusterConfig.getLocationConstraint().length);
+            InputFormat inputFormat = org.apache.hadoop.util.ReflectionUtils.newInstance(job.getInputFormatClass(),
+                    job.getConfiguration());
+            splits = inputFormat.getSplits(tmpJob);
             LOGGER.info("number of splits: " + splits.size());
             for (InputSplit split : splits)
                 LOGGER.info(split.toString());
@@ -692,6 +708,16 @@
                 readSchedule, new KeyValueParserFactory());
         ClusterConfig.setLocationConstraint(spec, scanner);
 
+        /** construct the sort operator to sort message states */
+        int[] keyFields = new int[] { 0 };
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
+        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
+        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastCheckpointedIteration,
+                WritableComparator.get(vertexIdClass).getClass());
+        ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
+                nkmFactory, sortCmpFactories, recordDescriptor);
+        ClusterConfig.setLocationConstraint(spec, sort);
+
         /**
          * construct the materializing write operator
          */
@@ -701,7 +727,7 @@
 
         /** construct runtime hook */
         RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
-                new RecoveryRuntimeHookFactory(jobId, lastCheckpointedIteration + 1, new ConfigurationFactory(
+                new RecoveryRuntimeHookFactory(jobId, lastCheckpointedIteration, new ConfigurationFactory(
                         pregelixJob.getConfiguration())));
         ClusterConfig.setLocationConstraint(spec, postSuperStep);
 
@@ -714,7 +740,8 @@
          */
         ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0,
-                materialize, 0);
+                sort, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
         spec.setFrameSize(frameSize);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index 41887c0..9389f62 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -21,15 +21,17 @@
 import java.util.List;
 import java.util.logging.Logger;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -49,17 +51,12 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs2.dataflow.ConfFactory;
 import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSWriteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.pregelix.api.graph.MsgList;
-import edu.uci.ics.pregelix.api.io.VertexInputFormat;
-import edu.uci.ics.pregelix.api.io.internal.InternalVertexInputFormat;
-import edu.uci.ics.pregelix.api.io.internal.InternalVertexOutputFormat;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.core.data.TypeTraits;
@@ -71,8 +68,8 @@
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.EmptyTupleSourceOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.FinalAggregateOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.HDFSFileWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.KeyValueParserFactory;
-import edu.uci.ics.pregelix.dataflow.KeyValueWriterFactory;
 import edu.uci.ics.pregelix.dataflow.MaterializingReadOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
@@ -344,7 +341,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+                true);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
         /**
@@ -525,14 +523,10 @@
     /** generate plan specific state checkpointing */
     protected JobSpecification[] generateStateCheckpointing(int lastSuccessfulIteration) throws HyracksException {
         JobSpecification[] msgCkpSpecs = super.generateStateCheckpointing(lastSuccessfulIteration);
-        PregelixJob tmpJob = this.createCloneJob("Vertex checkpointing for job " + jobId, pregelixJob);
-        tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
 
         /** generate secondary index checkpoint */
-        String checkpointPath = "/tmp/ckpoint/" + jobId + "/secondary/" + lastSuccessfulIteration;
-        FileOutputFormat.setOutputPath(tmpJob, new Path(checkpointPath));
-        tmpJob.setOutputKeyClass(BspUtils.getVertexIndexClass(tmpJob.getConfiguration()));
-        tmpJob.setOutputValueClass(MsgList.class);
+        PregelixJob tmpJob = this.createCloneJob("Secondary index checkpointing for job " + jobId, pregelixJob);
+
         JobSpecification secondaryBTreeCkp = generateSecondaryBTreeCheckpoint(lastSuccessfulIteration, tmpJob);
 
         JobSpecification[] specs = new JobSpecification[msgCkpSpecs.length + 1];
@@ -549,11 +543,12 @@
     @Override
     protected JobSpecification[] generateStateCheckpointLoading(int lastSuccessfulIteration, PregelixJob job)
             throws HyracksException {
-        JobSpecification[] msgCkpSpecs = generateStateCheckpointLoading(lastSuccessfulIteration, job);
-        PregelixJob tmpJob = this.createCloneJob("Vertex checkpointing for job " + jobId, pregelixJob);
-        tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
+        /** generate message checkpoint load */
+        JobSpecification[] msgCkpSpecs = super.generateStateCheckpointLoading(lastSuccessfulIteration, job);
 
         /** generate secondary index checkpoint load */
+        PregelixJob tmpJob = this.createCloneJob("Secondary index checkpoint loading for job " + jobId, pregelixJob);
+        tmpJob.setOutputFormatClass(SequenceFileOutputFormat.class);
         JobSpecification secondaryBTreeCkpLoad = generateSecondaryBTreeCheckpointLoad(lastSuccessfulIteration, tmpJob);
         JobSpecification[] specs = new JobSpecification[msgCkpSpecs.length + 1];
         for (int i = 0; i < msgCkpSpecs.length; i++) {
@@ -569,23 +564,23 @@
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(job.getConfiguration());
         JobSpecification spec = new JobSpecification();
 
-        String checkpointPath = "/tmp/ckpoint/" + jobId + "/secondary/" + lastSuccessfulIteration;;
+        String checkpointPath = BspUtils.getSecondaryIndexCheckpointPath(conf, lastSuccessfulIteration);
         PregelixJob tmpJob = createCloneJob("State checkpoint loading for job " + jobId, job);
-        tmpJob.setVertexInputFormatClass(InternalVertexInputFormat.class);
+        tmpJob.setInputFormatClass(SequenceFileInputFormat.class);
         try {
             FileInputFormat.setInputPaths(tmpJob, checkpointPath);
         } catch (IOException e) {
             throw new HyracksException(e);
         }
-        Configuration conf = job.getConfiguration();
 
         /***
-         * construct HDFS read operator
+         * HDFS read operator
          */
-        VertexInputFormat inputFormat = BspUtils.createVertexInputFormat(conf);
         List<InputSplit> splits = new ArrayList<InputSplit>();
         try {
-            splits = inputFormat.getSplits(tmpJob, ClusterConfig.getLocationConstraint().length);
+            InputFormat inputFormat = org.apache.hadoop.util.ReflectionUtils.newInstance(job.getInputFormatClass(),
+                    job.getConfiguration());
+            splits = inputFormat.getSplits(tmpJob);
             LOGGER.info("number of splits: " + splits.size());
             for (InputSplit split : splits)
                 LOGGER.info(split.toString());
@@ -599,6 +594,16 @@
                 readSchedule, new KeyValueParserFactory());
         ClusterConfig.setLocationConstraint(spec, scanner);
 
+        /** construct the sort operator to sort message states */
+        int[] keyFields = new int[] { 0 };
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
+        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
+        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastSuccessfulIteration,
+                WritableComparator.get(vertexIdClass).getClass());
+        ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
+                nkmFactory, sortCmpFactories, recordDescriptor);
+        ClusterConfig.setLocationConstraint(spec, sort);
+
         /**
          * construct bulk-load index operator
          */
@@ -621,8 +626,8 @@
          * connect operator descriptors
          */
         ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0,
-                btreeBulkLoad, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sort, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, btreeBulkLoad, 0);
         spec.setFrameSize(frameSize);
 
         return spec;
@@ -631,6 +636,12 @@
     @SuppressWarnings({ "rawtypes", "unchecked" })
     private JobSpecification generateSecondaryBTreeCheckpoint(int lastSuccessfulIteration, PregelixJob job)
             throws HyracksException {
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        String checkpointPath = BspUtils.getSecondaryIndexCheckpointPath(conf, lastSuccessfulIteration);
+        FileOutputFormat.setOutputPath(job, new Path(checkpointPath));
+        job.setOutputKeyClass(BspUtils.getVertexIndexClass(job.getConfiguration()));
+        job.setOutputValueClass(MsgList.class);
+
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(job.getConfiguration());
         Class<? extends Writable> msgListClass = MsgList.class;
         String readFile = lastSuccessfulIteration % 2 == 0 ? SECONDARY_INDEX_EVEN : SECONDARY_INDEX_ODD;
@@ -654,7 +665,6 @@
         /**
          * construct btree search operator
          */
-        ConfFactory confFactory = new ConfFactory(job);
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), msgListClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
@@ -674,8 +684,9 @@
         /**
          * construct write file operator
          */
-        HDFSWriteOperatorDescriptor writer = new HDFSWriteOperatorDescriptor(spec, job, new KeyValueWriterFactory(
-                confFactory));
+        IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                vertexIdClass.getName(), MsgList.class.getName());
+        HDFSFileWriteOperatorDescriptor writer = new HDFSFileWriteOperatorDescriptor(spec, job, inputRdFactory);
         ClusterConfig.setLocationConstraint(spec, writer);
 
         /**
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
index c29ea18..287b797 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
@@ -299,7 +299,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+                true);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
         /**
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
index dc61971..3b3c9e7 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
@@ -288,7 +288,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+                true);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
         /**
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
index 34f723f..e334095 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
@@ -302,7 +302,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+                true);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
         /**
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java
index ea6cc8a..89fbdcd 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java
@@ -20,12 +20,15 @@
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
+import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -52,6 +55,8 @@
     private static Map<String, List<String>> ipToNcMapping;
     private static String[] stores;
     private static Scheduler hdfsScheduler;
+    private static Set<String> blackListNodes = new HashSet<String>();
+    private static IHyracksClientConnection hcc;
 
     /**
      * let tests set config path to be whatever
@@ -197,9 +202,19 @@
 
     public static void loadClusterConfig(String ipAddress, int port) throws HyracksException {
         try {
-            IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+            if (hcc == null) {
+                hcc = new HyracksConnection(ipAddress, port);
+            }
             Map<String, NodeControllerInfo> ncNameToNcInfos = new TreeMap<String, NodeControllerInfo>();
             ncNameToNcInfos.putAll(hcc.getNodeControllerInfos());
+
+            /**
+             * remove black list nodes -- which had disk failures
+             */
+            for (String blackListNode : blackListNodes) {
+                ncNameToNcInfos.remove(blackListNode);
+            }
+
             NCs = new String[ncNameToNcInfos.size()];
             ipToNcMapping = new HashMap<String, List<String>>();
             int i = 0;
@@ -216,7 +231,7 @@
                 i++;
             }
 
-            hdfsScheduler = new Scheduler(ipAddress, port);
+            hdfsScheduler = new Scheduler(hcc.getNodeControllerInfos(), hcc.getClusterTopology());
         } catch (Exception e) {
             throw new IllegalStateException(e);
         }
@@ -240,4 +255,8 @@
         }
         return locations;
     }
+
+    public static void addToBlackListNodes(Collection<String> nodes) {
+        blackListNodes.addAll(nodes);
+    }
 }
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java
new file mode 100644
index 0000000..a4c4501
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.core.util;
+
+import java.io.IOException;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The util to analysis exceptions
+ * 
+ * @author yingyib
+ */
+public class ExceptionUtilities {
+
+    /**
+     * Check whether a exception is recoverable or not
+     * 
+     * @param exception
+     * @return true or false
+     */
+    public static boolean recoverable(Exception exception, Set<String> blackListNodes) {
+        String message = exception.getMessage();
+
+        /***
+         * check interrupted exception
+         */
+        if (exception instanceof InterruptedException || (message.contains("Node") && message.contains("not live"))
+                || message.contains("Failure occurred on input")) {
+            return true;
+        }
+        Throwable cause = exception;
+        while ((cause = cause.getCause()) != null) {
+            if (cause instanceof InterruptedException) {
+                return true;
+            }
+        }
+
+        /***
+         * check io exception
+         */
+        cause = exception;
+        String blackListNode = null;
+        if (cause instanceof HyracksDataException) {
+            blackListNode = ((HyracksDataException) cause).getNodeId();
+        }
+        while ((cause = cause.getCause()) != null) {
+            if (cause instanceof IOException) {
+                if (containsIOManager(cause)) {
+                    if (blackListNode != null) {
+                        blackListNodes.add(blackListNode);
+                    }
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Check if the exception traces contains the IOManager, which means there are disk failures
+     * 
+     * @param cause
+     * @return true if IOManager is in the trace; false otherwise.
+     */
+    private static boolean containsIOManager(Throwable cause) {
+        StackTraceElement[] traces = cause.getStackTrace();
+        for (StackTraceElement e : traces) {
+            if (e.getClassName().endsWith("IOManager")) {
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
index e1795de..aabd4ba 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
@@ -61,8 +61,8 @@
         ccConfig.defaultMaxJobAttempts = 0;
         ccConfig.jobHistorySize = 1;
         ccConfig.profileDumpPeriod = -1;
-        //ccConfig.heartbeatPeriod = 5000;
-        //ccConfig.maxHeartbeatLapsePeriods = 1;
+        ccConfig.heartbeatPeriod = 50;
+        ccConfig.maxHeartbeatLapsePeriods = 15;
 
         // cluster controller
         cc = new ClusterControllerService(ccConfig);
@@ -98,14 +98,22 @@
         ClusterConfig.loadClusterConfig(CC_HOST, TEST_HYRACKS_CC_CLIENT_PORT);
     }
 
-    public static void showDownNC1() throws Exception {
+    public static void startNC1() throws Exception {
+        nc1.start();
+    }
+
+    public static void shutdownNC1() throws Exception {
         nc1.stop();
     }
 
-    public static void showDownNC2() throws Exception {
+    public static void shutdownNC2() throws Exception {
         nc2.stop();
     }
 
+    public static void shutdownCC() throws Exception {
+        cc.stop();
+    }
+
     public static void deinit() throws Exception {
         nc2.stop();
         nc1.stop();
diff --git a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
index 2a6cab2..d7a0ead 100644
--- a/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
+++ b/pregelix/pregelix-core/src/main/resources/scripts/startcc.sh
@@ -58,8 +58,8 @@
 
 if [ -f "conf/topology.xml"  ]; then
 #Launch hyracks cc script with topology
-${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -heartbeat-period 5000 -max-heartbeat-lapse-periods 4 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
 else
 #Launch hyracks cc script without toplogy
-${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -heartbeat-period 5000 -max-heartbeat-lapse-periods 4 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
 fi
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
index 7221cb5..b22e468 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
@@ -230,8 +230,16 @@
 
     @Override
     public void fail() throws HyracksDataException {
-        for (IFrameWriter writer : writers)
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        } finally {
+            treeIndexOpHelper.close();
+        }
+        for (IFrameWriter writer : writers) {
             writer.fail();
+        }
     }
 
     @Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
index b21cd2a..0ecfd03 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
@@ -254,8 +254,16 @@
 
     @Override
     public void fail() throws HyracksDataException {
-        for (IFrameWriter writer : writers)
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        } finally {
+            treeIndexOpHelper.close();
+        }
+        for (IFrameWriter writer : writers) {
             writer.fail();
+        }
     }
 
     /** compare tuples */
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
index dd6ee3c..e64e9cc 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
@@ -245,6 +245,13 @@
 
     @Override
     public void fail() throws HyracksDataException {
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        } finally {
+            treeIndexOpHelper.close();
+        }
         writer.fail();
     }
 
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
index 3cebfb8..a9c787f 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
@@ -219,8 +219,16 @@
 
     @Override
     public void fail() throws HyracksDataException {
-        for (IFrameWriter writer : writers)
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        } finally {
+            treeIndexOpHelper.close();
+        }
+        for (IFrameWriter writer : writers) {
             writer.fail();
+        }
     }
 
     /** compare tuples */
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
index bbe2764..86a211f 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
@@ -205,6 +205,13 @@
 
     @Override
     public void fail() throws HyracksDataException {
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        } finally {
+            treeIndexOpHelper.close();
+        }
         writer.fail();
     }
 
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
index c4890e1..c985f64 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
@@ -100,6 +100,11 @@
 
     @Override
     public void fail() throws HyracksDataException {
-
+        try {
+            bulkLoader.end();
+        } catch (IndexException e) {
+            treeIndexOpHelper.close();
+            throw new HyracksDataException(e);
+        } 
     }
 }
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
index bd85e3e..de87909 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
@@ -224,8 +224,16 @@
 
     @Override
     public void fail() throws HyracksDataException {
-        for (IFrameWriter writer : writers)
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        } finally {
+            treeIndexHelper.close();
+        }
+        for (IFrameWriter writer : writers) {
             writer.fail();
+        }
     }
 
     @Override
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
index 9a21680..a1177c8 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
@@ -19,42 +19,45 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
 
 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.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
 import edu.uci.ics.hyracks.hdfs.ContextFactory;
-import edu.uci.ics.pregelix.api.graph.Vertex;
-import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
-import edu.uci.ics.pregelix.api.io.VertexWriter;
-import edu.uci.ics.pregelix.api.util.BspUtils;
-import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+import edu.uci.ics.hyracks.hdfs2.dataflow.ConfFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 
 public class HDFSFileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
     private static final long serialVersionUID = 1L;
-    private final IConfigurationFactory confFactory;
+    private final ConfFactory confFactory;
     private final IRecordDescriptorFactory inputRdFactory;
 
-    public HDFSFileWriteOperatorDescriptor(JobSpecification spec, IConfigurationFactory confFactory,
-            IRecordDescriptorFactory inputRdFactory) {
+    public HDFSFileWriteOperatorDescriptor(JobSpecification spec, Job conf, IRecordDescriptorFactory inputRdFactory)
+            throws HyracksException {
         super(spec, 1, 0);
-        this.confFactory = confFactory;
-        this.inputRdFactory = inputRdFactory;
+        try {
+            this.confFactory = new ConfFactory(conf);
+            this.inputRdFactory = inputRdFactory;
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
     }
 
     @SuppressWarnings("rawtypes")
@@ -65,12 +68,12 @@
         return new AbstractUnaryInputSinkOperatorNodePushable() {
             private RecordDescriptor rd0;
             private FrameDeserializer frameDeserializer;
-            private Configuration conf;
-            private VertexWriter vertexWriter;
+            private Job job;
+            private RecordWriter recordWriter;
             private TaskAttemptContext context;
+            private ContextFactory ctxFactory = new ContextFactory();
             private String TEMP_DIR = "_temporary";
             private ClassLoader ctxCL;
-            private ContextFactory ctxFactory = new ContextFactory();
 
             @Override
             public void open() throws HyracksDataException {
@@ -79,16 +82,16 @@
                 frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
                 ctxCL = Thread.currentThread().getContextClassLoader();
                 Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
-                conf = confFactory.createConfiguration(ctx);
-
-                VertexOutputFormat outputFormat = BspUtils.createVertexOutputFormat(conf);
-                context = ctxFactory.createContext(conf, partition);
-                context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
+                job = confFactory.getConf();
                 try {
-                    vertexWriter = outputFormat.createVertexWriter(context);
+                    OutputFormat outputFormat = ReflectionUtils.newInstance(job.getOutputFormatClass(),
+                            job.getConfiguration());
+                    context = ctxFactory.createContext(job.getConfiguration(), partition);
+                    context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
+                    recordWriter = outputFormat.getRecordWriter(context);
                 } catch (InterruptedException e) {
                     throw new HyracksDataException(e);
-                } catch (IOException e) {
+                } catch (Exception e) {
                     throw new HyracksDataException(e);
                 }
             }
@@ -100,8 +103,9 @@
                 try {
                     while (!frameDeserializer.done()) {
                         Object[] tuple = frameDeserializer.deserializeRecord();
-                        Vertex value = (Vertex) tuple[1];
-                        vertexWriter.writeVertex(value);
+                        Object key = tuple[0];
+                        Object value = tuple[1];
+                        recordWriter.write(key, value);
                     }
                 } catch (InterruptedException e) {
                     throw new HyracksDataException(e);
@@ -118,7 +122,7 @@
             @Override
             public void close() throws HyracksDataException {
                 try {
-                    vertexWriter.close(context);
+                    recordWriter.close(context);
                     moveFilesToFinalPath();
                 } catch (InterruptedException e) {
                     throw new HyracksDataException(e);
@@ -129,9 +133,8 @@
 
             private void moveFilesToFinalPath() throws HyracksDataException {
                 try {
-                    JobContext job = ctxFactory.createJobContext(conf);
                     Path outputPath = FileOutputFormat.getOutputPath(job);
-                    FileSystem dfs = FileSystem.get(conf);
+                    FileSystem dfs = FileSystem.get(job.getConfiguration());
                     Path filePath = new Path(outputPath, "part-" + new Integer(partition).toString());
                     FileStatus[] results = findPartitionPaths(outputPath, dfs);
                     if (results.length >= 1) {
@@ -161,7 +164,7 @@
                 FileStatus[] tempPaths = dfs.listStatus(outputPath, new PathFilter() {
                     @Override
                     public boolean accept(Path dir) {
-                        return dir.getName().endsWith(TEMP_DIR);
+                        return dir.getName().endsWith(TEMP_DIR) && dir.getName().indexOf(".crc") < 0;
                     }
                 });
                 Path tempDir = tempPaths[0].getPath();
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
index ca8f190..b44b643 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
@@ -30,9 +30,12 @@
 
 public class MaterializingReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
     private static final long serialVersionUID = 1L;
+    private final boolean removeIterationState;
 
-    public MaterializingReadOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor) {
+    public MaterializingReadOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor,
+            boolean removeIterationState) {
         super(spec, 1, 1);
+        this.removeIterationState = removeIterationState;
         recordDescriptors[0] = recordDescriptor;
     }
 
@@ -73,7 +76,7 @@
 
             @Override
             public void fail() throws HyracksDataException {
-
+                writer.fail();
             }
 
             @Override
@@ -81,7 +84,9 @@
                 /**
                  * remove last iteration's state
                  */
-                IterationUtils.removeIterationState(ctx, partition);
+                if (removeIterationState) {
+                    IterationUtils.removeIterationState(ctx, partition);
+                }
                 writer.close();
                 complete = true;
             }
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java
new file mode 100644
index 0000000..f3ec40e
--- /dev/null
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java
@@ -0,0 +1,192 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.dataflow;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+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.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.ContextFactory;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
+import edu.uci.ics.pregelix.api.io.VertexWriter;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
+
+public class VertexFileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private final IConfigurationFactory confFactory;
+    private final IRecordDescriptorFactory inputRdFactory;
+
+    public VertexFileWriteOperatorDescriptor(JobSpecification spec, IConfigurationFactory confFactory,
+            IRecordDescriptorFactory inputRdFactory) {
+        super(spec, 1, 0);
+        this.confFactory = confFactory;
+        this.inputRdFactory = inputRdFactory;
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+            throws HyracksDataException {
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+            private RecordDescriptor rd0;
+            private FrameDeserializer frameDeserializer;
+            private Configuration conf;
+            private VertexWriter vertexWriter;
+            private TaskAttemptContext context;
+            private String TEMP_DIR = "_temporary";
+            private ClassLoader ctxCL;
+            private ContextFactory ctxFactory = new ContextFactory();
+
+            @Override
+            public void open() throws HyracksDataException {
+                rd0 = inputRdFactory == null ? recordDescProvider.getInputRecordDescriptor(getActivityId(), 0)
+                        : inputRdFactory.createRecordDescriptor(ctx);
+                frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
+                ctxCL = Thread.currentThread().getContextClassLoader();
+                Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+                conf = confFactory.createConfiguration(ctx);
+
+                VertexOutputFormat outputFormat = BspUtils.createVertexOutputFormat(conf);
+                context = ctxFactory.createContext(conf, partition);
+                context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
+                try {
+                    vertexWriter = outputFormat.createVertexWriter(context);
+                } catch (InterruptedException e) {
+                    throw new HyracksDataException(e);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @SuppressWarnings("unchecked")
+            @Override
+            public void nextFrame(ByteBuffer frame) throws HyracksDataException {
+                frameDeserializer.reset(frame);
+                try {
+                    while (!frameDeserializer.done()) {
+                        Object[] tuple = frameDeserializer.deserializeRecord();
+                        Vertex value = (Vertex) tuple[1];
+                        vertexWriter.writeVertex(value);
+                    }
+                } catch (InterruptedException e) {
+                    throw new HyracksDataException(e);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                Thread.currentThread().setContextClassLoader(ctxCL);
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                try {
+                    vertexWriter.close(context);
+                    moveFilesToFinalPath();
+                } catch (InterruptedException e) {
+                    throw new HyracksDataException(e);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            private void moveFilesToFinalPath() throws HyracksDataException {
+                try {
+                    JobContext job = ctxFactory.createJobContext(conf);
+                    Path outputPath = FileOutputFormat.getOutputPath(job);
+                    FileSystem dfs = FileSystem.get(conf);
+                    Path filePath = new Path(outputPath, "part-" + new Integer(partition).toString());
+                    FileStatus[] results = findPartitionPaths(outputPath, dfs);
+                    if (results.length >= 1) {
+                        /**
+                         * for Hadoop-0.20.2
+                         */
+                        renameFile(dfs, filePath, results);
+                    } else {
+                        /**
+                         * for Hadoop-0.23.1
+                         */
+                        int jobId = job.getJobID().getId();
+                        outputPath = new Path(outputPath.toString() + File.separator + TEMP_DIR + File.separator
+                                + jobId);
+                        results = findPartitionPaths(outputPath, dfs);
+                        renameFile(dfs, filePath, results);
+                    }
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                } finally {
+                    Thread.currentThread().setContextClassLoader(ctxCL);
+                }
+            }
+
+            private FileStatus[] findPartitionPaths(Path outputPath, FileSystem dfs) throws FileNotFoundException,
+                    IOException {
+                FileStatus[] tempPaths = dfs.listStatus(outputPath, new PathFilter() {
+                    @Override
+                    public boolean accept(Path dir) {
+                        return dir.getName().endsWith(TEMP_DIR) && dir.getName().indexOf(".crc") < 0;
+                    }
+                });
+                Path tempDir = tempPaths[0].getPath();
+                FileStatus[] results = dfs.listStatus(tempDir, new PathFilter() {
+                    @Override
+                    public boolean accept(Path dir) {
+                        return dir.getName().indexOf(context.getTaskAttemptID().toString()) >= 0
+                                && dir.getName().indexOf(".crc") < 0;
+                    }
+                });
+                return results;
+            }
+
+            private void renameFile(FileSystem dfs, Path filePath, FileStatus[] results) throws IOException,
+                    HyracksDataException, FileNotFoundException {
+                Path srcDir = results[0].getPath();
+                if (!dfs.exists(srcDir))
+                    throw new HyracksDataException("file " + srcDir.toString() + " does not exist!");
+
+                FileStatus[] srcFiles = dfs.listStatus(srcDir);
+                Path srcFile = srcFiles[0].getPath();
+                dfs.delete(filePath, true);
+                dfs.rename(srcFile, filePath);
+            }
+
+        };
+    }
+}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
index bfe89ab..f3f7513 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
@@ -147,11 +147,15 @@
         return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
     }
 
-    public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges, int currentIteration) {
+    public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges, long currentIteration) {
         Boolean toMove = jobIdToMove.get(jobId);
         if (toMove == null || toMove == true) {
             if (jobIdToSuperStep.get(jobId) == null) {
-                jobIdToSuperStep.put(jobId, 0L);
+                if (currentIteration <= 0) {
+                    jobIdToSuperStep.put(jobId, 0L);
+                } else {
+                    jobIdToSuperStep.put(jobId, currentIteration);
+                }
             }
 
             long superStep = jobIdToSuperStep.get(jobId);
@@ -175,6 +179,35 @@
         System.gc();
     }
 
+    public synchronized void recoverVertexProperties(String jobId, long numVertices, long numEdges,
+            long currentIteration) {
+        if (jobIdToSuperStep.get(jobId) == null) {
+            if (currentIteration <= 0) {
+                jobIdToSuperStep.put(jobId, 0L);
+            } else {
+                jobIdToSuperStep.put(jobId, currentIteration);
+            }
+        }
+
+        long superStep = jobIdToSuperStep.get(jobId);
+        List<FileReference> files = iterationToFiles.remove(superStep - 1);
+        if (files != null) {
+            for (FileReference fileRef : files)
+                fileRef.delete();
+        }
+
+        if (currentIteration > 0) {
+            Vertex.setSuperstep(currentIteration);
+        } else {
+            Vertex.setSuperstep(++superStep);
+        }
+        Vertex.setNumVertices(numVertices);
+        Vertex.setNumEdges(numEdges);
+        jobIdToSuperStep.put(jobId, superStep);
+        jobIdToMove.put(jobId, true);
+        LOGGER.info("recovered iteration " + Vertex.getSuperstep());
+    }
+
     public synchronized void endSuperStep(String pregelixJobId) {
         jobIdToMove.put(pregelixJobId, true);
         LOGGER.info("end iteration " + Vertex.getSuperstep());
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
index 1cf81ac..02097bf 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
@@ -75,13 +75,21 @@
         context.endSuperStep(giraphJobId);
     }
 
-    public static void setProperties(String jobId, IHyracksTaskContext ctx, Configuration conf, int currentIteration) {
+    public static void setProperties(String jobId, IHyracksTaskContext ctx, Configuration conf, long currentIteration) {
         INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
         RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
         context.setVertexProperties(jobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
                 conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration);
     }
 
+    public static void recoverProperties(String jobId, IHyracksTaskContext ctx, Configuration conf,
+            long currentIteration) {
+        INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
+        RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
+        context.recoverVertexProperties(jobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
+                conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration);
+    }
+
     public static void writeTerminationState(Configuration conf, String jobId, boolean terminate)
             throws HyracksDataException {
         try {
diff --git a/pregelix/pregelix-dist/pom.xml b/pregelix/pregelix-dist/pom.xml
index f0551a6..cec6efe 100644
--- a/pregelix/pregelix-dist/pom.xml
+++ b/pregelix/pregelix-dist/pom.xml
@@ -63,5 +63,10 @@
 			<artifactId>pregelix-example</artifactId>
 			<version>0.2.10-SNAPSHOT</version>
 		</dependency>
+		<dependency>
+                        <groupId>edu.uci.ics.hyracks</groupId>
+                        <artifactId>pregelix-benchmark</artifactId>
+                        <version>0.2.10-SNAPSHOT</version>
+                </dependency>
 	</dependencies>
 </project>
diff --git a/pregelix/pregelix-dist/src/main/resources/scripts/startcc.sh b/pregelix/pregelix-dist/src/main/resources/scripts/startcc.sh
index 2a6cab2..d7a0ead 100644
--- a/pregelix/pregelix-dist/src/main/resources/scripts/startcc.sh
+++ b/pregelix/pregelix-dist/src/main/resources/scripts/startcc.sh
@@ -58,8 +58,8 @@
 
 if [ -f "conf/topology.xml"  ]; then
 #Launch hyracks cc script with topology
-${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -heartbeat-period 5000 -max-heartbeat-lapse-periods 4 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
 else
 #Launch hyracks cc script without toplogy
-${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -heartbeat-period 5000 -max-heartbeat-lapse-periods 4 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
 fi
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java
index f99321a..393c8c9 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java
@@ -45,7 +45,7 @@
         public String ipAddress;
 
         @Option(name = "-port", usage = "port of cluster controller", required = false)
-        public int port;
+        public int port = 3099;
 
         @Option(name = "-plan", usage = "query plan choice", required = false)
         public Plan planChoice = Plan.OUTER_JOIN;
@@ -67,6 +67,9 @@
 
         @Option(name = "-runtime-profiling", usage = "whether to do runtime profifling", required = false)
         public String profiling = "false";
+
+        @Option(name = "-ckp-interval", usage = "checkpointing interval -- for fault-tolerance", required = false)
+        public int ckpInterval = -1;
     }
 
     public static void run(String[] args, PregelixJob job) throws Exception {
@@ -125,6 +128,7 @@
         job.getConfiguration().setLong(ReachabilityVertex.DEST_ID, options.destId);
         if (options.numIteration > 0)
             job.getConfiguration().setLong(PageRankVertex.ITERATIONS, options.numIteration);
+        job.setCheckpointingInterval(options.ckpInterval);
     }
 
 }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java
index 90065c2..44d23a4 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java
@@ -14,13 +14,8 @@
  */
 package edu.uci.ics.pregelix.example.data;
 
-import java.io.DataInput;
-import java.io.DataInputStream;
-
-import org.apache.hadoop.io.WritableUtils;
-
 import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
-import edu.uci.ics.pregelix.api.util.ResetableByteArrayInputStream;
+import edu.uci.ics.pregelix.example.utils.SerDeUtils;
 
 /**
  * @author yingyib
@@ -31,14 +26,10 @@
     private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
     private static final int NEGATIVE_LONG_MASK = (0 << 30);
 
-    private ResetableByteArrayInputStream bis = new ResetableByteArrayInputStream();
-    private DataInput dis = new DataInputStream(bis);
-
     @Override
     public int getNormalizedKey(byte[] bytes, int start, int length) {
         try {
-            bis.setByteArray(bytes, start);
-            long value = WritableUtils.readVLong(dis);
+            long value = SerDeUtils.readVLong(bytes, start, length);
             int highValue = (int) (value >> 32);
             if (highValue > 0) {
                 /**
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java
index 1c5f629..ffbbff4 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java
@@ -16,14 +16,12 @@
 package edu.uci.ics.pregelix.example.io;
 
 import java.io.DataInput;
-import java.io.DataInputStream;
 
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.io.WritableUtils;
 
 import edu.uci.ics.pregelix.api.io.WritableSizable;
-import edu.uci.ics.pregelix.api.util.ResetableByteArrayInputStream;
+import edu.uci.ics.pregelix.example.utils.SerDeUtils;
 
 /**
  * A WritableComparable for longs in a variable-length format. Such values take
@@ -65,8 +63,6 @@
 
     /** A Comparator optimized for LongWritable. */
     public static class Comparator extends WritableComparator {
-        private ResetableByteArrayInputStream bis = new ResetableByteArrayInputStream();
-        private DataInput dis = new DataInputStream(bis);
 
         public Comparator() {
             super(VLongWritable.class);
@@ -74,10 +70,8 @@
 
         public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
             try {
-                bis.setByteArray(b1, s1);
-                long thisValue = WritableUtils.readVLong(dis);
-                bis.setByteArray(b2, s2);
-                long thatValue = WritableUtils.readVLong(dis);
+                long thisValue = SerDeUtils.readVLong(b1, s1, l1);
+                long thatValue = SerDeUtils.readVLong(b2, s2, l2);
                 return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
             } catch (Exception e) {
                 throw new IllegalStateException(e);
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java
new file mode 100644
index 0000000..2800187
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java
@@ -0,0 +1,56 @@
+package edu.uci.ics.pregelix.example.utils;
+
+import java.io.IOException;
+
+public class SerDeUtils {
+
+    /**
+     * Reads a zero-compressed encoded long from input stream and returns it.
+     * 
+     * @param stream
+     *            Binary input stream
+     * @throws java.io.IOException
+     * @return deserialized long from stream.
+     */
+    public static long readVLong(byte[] data, int start, int length) throws IOException {
+        byte firstByte = data[start];
+        int len = decodeVIntSize(firstByte);
+        if (len == 1) {
+            return firstByte;
+        }
+        long i = 0;
+        for (int idx = 0; idx < len - 1; idx++) {
+            i = i << 8;
+            i = i | (data[++start] & 0xFF);
+        }
+        return (isNegativeVInt(firstByte) ? (i ^ -1L) : i);
+    }
+
+    /**
+     * Parse the first byte of a vint/vlong to determine the number of bytes
+     * 
+     * @param value
+     *            the first byte of the vint/vlong
+     * @return the total number of bytes (1 to 9)
+     */
+    public static int decodeVIntSize(byte value) {
+        if (value >= -112) {
+            return 1;
+        } else if (value < -120) {
+            return -119 - value;
+        }
+        return -111 - value;
+    }
+
+    /**
+     * Given the first byte of a vint/vlong, determine the sign
+     * 
+     * @param value
+     *            the first byte
+     * @return is the value negative
+     */
+    public static boolean isNegativeVInt(byte value) {
+        return value < -120 || (value >= -112 && value < 0);
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java
new file mode 100644
index 0000000..efc7bcc
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.example;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Test;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
+import edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.ConnectedComponentsVertex.SimpleConnectedComponentsVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * @author yingyib
+ */
+public class FailureRecoveryConnectedComponentsTest {
+    private static String INPUTPATH = "data/webmapcomplex";
+    private static String OUTPUTPAH = "actual/result";
+    private static String EXPECTEDPATH = "src/test/resources/expected/ConnectedComponentsRealComplex2";
+
+    @Test
+    public void test() throws Exception {
+        TestCluster testCluster = new TestCluster();
+        try {
+            PregelixJob job = new PregelixJob(ConnectedComponentsVertex.class.getName());
+            job.setVertexClass(ConnectedComponentsVertex.class);
+            job.setVertexClass(ConnectedComponentsVertex.class);
+            job.setVertexInputFormatClass(TextConnectedComponentsInputFormat.class);
+            job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
+            job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
+            job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+            job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
+            job.setDynamicVertexValueSize(true);
+            FileInputFormat.setInputPaths(job, INPUTPATH);
+            FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+            job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
+            job.setCheckpointHook(ConservativeCheckpointHook.class);
+
+            testCluster.setUp();
+            Driver driver = new Driver(PageRankVertex.class);
+            Thread thread = new Thread(new Runnable() {
+
+                @Override
+                public void run() {
+                    try {
+                        synchronized (this) {
+                            while (Vertex.getSuperstep() <= 5) {
+                                this.wait(200);
+                            }
+                            PregelixHyracksIntegrationUtil.shutdownNC1();
+                        }
+                    } catch (Exception e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+            });
+            thread.start();
+            driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+
+            TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+        } catch (Exception e) {
+            PregelixHyracksIntegrationUtil.shutdownNC2();
+            testCluster.cleanupHDFS();
+            throw e;
+        }
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java
new file mode 100644
index 0000000..ff1e29f
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.example;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Test;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
+import edu.uci.ics.pregelix.core.base.IDriver.Plan;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * @author yingyib
+ */
+public class FailureRecoveryInnerJoinTest {
+    private static String INPUTPATH = "data/webmap";
+    private static String OUTPUTPAH = "actual/result";
+    private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal2";
+
+    @Test
+    public void test() throws Exception {
+        TestCluster testCluster = new TestCluster();
+
+        try {
+            PregelixJob job = new PregelixJob(PageRankVertex.class.getName());
+            job.setVertexClass(PageRankVertex.class);
+            job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+            job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+            job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+            job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+            FileInputFormat.setInputPaths(job, INPUTPATH);
+            FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+            job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+            job.setCheckpointHook(ConservativeCheckpointHook.class);
+
+            testCluster.setUp();
+            Driver driver = new Driver(PageRankVertex.class);
+            Thread thread = new Thread(new Runnable() {
+
+                @Override
+                public void run() {
+                    try {
+                        synchronized (this) {
+                            while (Vertex.getSuperstep() <= 5) {
+                                this.wait(200);
+                            }
+                            PregelixHyracksIntegrationUtil.shutdownNC1();
+                        }
+                    } catch (Exception e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+            });
+            thread.start();
+            driver.runJob(job, Plan.INNER_JOIN, "127.0.0.1",
+                    PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
+
+            TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+        } catch (Exception e) {
+            PregelixHyracksIntegrationUtil.shutdownNC2();
+            testCluster.cleanupHDFS();
+            throw e;
+        }
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
index dac087f..3fdaf15 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
@@ -21,6 +21,7 @@
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.junit.Test;
 
+import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
 import edu.uci.ics.pregelix.core.driver.Driver;
@@ -37,7 +38,7 @@
 public class FailureRecoveryTest {
     private static String INPUTPATH = "data/webmap";
     private static String OUTPUTPAH = "actual/result";
-    private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal";
+    private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal2";
 
     @Test
     public void test() throws Exception {
@@ -57,29 +58,30 @@
 
             testCluster.setUp();
             Driver driver = new Driver(PageRankVertex.class);
-            //            Thread thread = new Thread(new Runnable() {
-            //
-            //                @Override
-            //                public void run() {
-            //                    try {
-            //                        synchronized (this) {
-            //                            this.wait(10000);
-            //                            PregelixHyracksIntegrationUtil.showDownNC1();
-            //                        }
-            //                    } catch (Exception e) {
-            //                        throw new IllegalStateException(e);
-            //                    }
-            //                }
-            //
-            //            });
-            //thread.start();
+            Thread thread = new Thread(new Runnable() {
+
+                @Override
+                public void run() {
+                    try {
+                        synchronized (this) {
+                            while (Vertex.getSuperstep() <= 5) {
+                                this.wait(200);
+                            }
+                            PregelixHyracksIntegrationUtil.shutdownNC1();
+                        }
+                    } catch (Exception e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+            });
+            thread.start();
             driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
 
             TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
         } catch (Exception e) {
+            PregelixHyracksIntegrationUtil.shutdownNC2();
+            testCluster.cleanupHDFS();
             throw e;
-        } finally {
-            testCluster.tearDown();
         }
     }
 
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java
new file mode 100644
index 0000000..e006ccd
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.example;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Test;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * @author yingyib
+ */
+public class FailureRecoveryWithoutCheckpointTest {
+    private static String INPUTPATH = "data/webmap";
+    private static String OUTPUTPAH = "actual/result";
+    private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal2";
+
+    @Test
+    public void test() throws Exception {
+        TestCluster testCluster = new TestCluster();
+
+        try {
+            PregelixJob job = new PregelixJob(PageRankVertex.class.getName());
+            job.setVertexClass(PageRankVertex.class);
+            job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+            job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+            job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+            job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+            FileInputFormat.setInputPaths(job, INPUTPATH);
+            FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+            job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+
+            testCluster.setUp();
+            Driver driver = new Driver(PageRankVertex.class);
+            Thread thread = new Thread(new Runnable() {
+
+                @Override
+                public void run() {
+                    try {
+                        synchronized (this) {
+                            while (Vertex.getSuperstep() <= 5) {
+                                this.wait(200);
+                            }
+                            PregelixHyracksIntegrationUtil.shutdownNC1();
+                        }
+                    } catch (Exception e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+            });
+            thread.start();
+            driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+
+            TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+        } catch (Exception e) {
+            PregelixHyracksIntegrationUtil.shutdownNC2();
+            testCluster.cleanupHDFS();
+            throw e;
+        }
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/FailureVertex.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureVertex.java
similarity index 100%
rename from pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/FailureVertex.java
rename to pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureVertex.java
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java
index 5a485ba..dc7a28d 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java
@@ -24,6 +24,7 @@
 import org.junit.Test;
 
 import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
 import edu.uci.ics.pregelix.core.driver.Driver;
 import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
 import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
@@ -55,7 +56,7 @@
             job1.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
             FileInputFormat.setInputPaths(job1, INPUTPATH);
             job1.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
-            //job1.setCheckpointHook(ConservativeCheckpointHook.class);
+            job1.setCheckpointHook(ConservativeCheckpointHook.class);
 
             PregelixJob job2 = new PregelixJob(PageRankVertex.class.getName());
             job2.setVertexClass(PageRankVertex.class);
@@ -65,7 +66,7 @@
             job2.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
             FileOutputFormat.setOutputPath(job2, new Path(OUTPUTPAH));
             job2.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
-            //job2.setCheckpointHook(ConservativeCheckpointHook.class);
+            job2.setCheckpointHook(ConservativeCheckpointHook.class);
 
             jobs.add(job1);
             jobs.add(job2);
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java
index 40ea690..660d9eb 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java
@@ -126,7 +126,7 @@
     /**
      * cleanup hdfs cluster
      */
-    private void cleanupHDFS() throws Exception {
+    public void cleanupHDFS() throws Exception {
         dfsCluster.shutdown();
     }
 
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0
new file mode 100755
index 0000000..2c975de
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0
@@ -0,0 +1,10 @@
+0	0
+2	0
+4	0
+6	0
+8	0
+10	0
+12	0
+14	0
+16	0
+18	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1
new file mode 100755
index 0000000..6976bc1
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1
@@ -0,0 +1,13 @@
+1	0
+3	0
+5	0
+7	0
+9	0
+11	0
+13	0
+15	0
+17	0
+19	0
+21	21
+25	25
+27	27
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal2/part-0 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal2/part-0
new file mode 100755
index 0000000..d135b86
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal2/part-0
@@ -0,0 +1,10 @@
+0	0.008290140026154316
+2	0.14646839195826472
+4	0.03976979906329426
+6	0.015736276824953852
+8	0.010628239626209894
+10	0.008290140026154316
+12	0.14646839195826472
+14	0.03976979906329426
+16	0.015736276824953852
+18	0.010628239626209894
diff --git a/pregelix/pregelix-example/src/test/resources/expected/PageRankReal2/part-1 b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal2/part-1
new file mode 100755
index 0000000..d3badee
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/resources/expected/PageRankReal2/part-1
@@ -0,0 +1,10 @@
+1	0.15351528192471647
+3	0.08125113985998214
+5	0.0225041581462058
+7	0.012542224114863661
+9	0.009294348455354817
+11	0.15351528192471647
+13	0.08125113985998214
+15	0.0225041581462058
+17	0.012542224114863661
+19	0.009294348455354817
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java
index 35e7cd8..4720272 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java
@@ -30,11 +30,11 @@
  */
 public class RecoveryRuntimeHookFactory implements IRuntimeHookFactory {
     private static final long serialVersionUID = 1L;
-    private final int currentSuperStep;
+    private final long currentSuperStep;
     private String jobId;
     private IConfigurationFactory confFactory;
 
-    public RecoveryRuntimeHookFactory(String jobId, int currentSuperStep, IConfigurationFactory confFactory) {
+    public RecoveryRuntimeHookFactory(String jobId, long currentSuperStep, IConfigurationFactory confFactory) {
         this.currentSuperStep = currentSuperStep;
         this.jobId = jobId;
         this.confFactory = confFactory;
@@ -48,7 +48,7 @@
             public void configure(IHyracksTaskContext ctx) throws HyracksDataException {
                 IterationUtils.endSuperStep(jobId, ctx);
                 Configuration conf = confFactory.createConfiguration(ctx);
-                IterationUtils.setProperties(jobId, ctx, conf, currentSuperStep);
+                IterationUtils.recoverProperties(jobId, ctx, conf, currentSuperStep);
             }
 
         };